From a029034d8c0a60ad760edeb00a43217e05cc3101 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 19 Nov 2018 09:14:36 +0300 Subject: [PATCH 01/47] CLICKHOUSE-4127: Fix ALTER of destination table for the BUFFER engine. --- dbms/src/Storages/StorageBuffer.cpp | 89 +++++++++++-------- ...r_destination_for_storage_buffer.reference | 19 ++++ ...3_alter_destination_for_storage_buffer.sql | 32 +++++++ 3 files changed, 103 insertions(+), 37 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.reference create mode 100644 dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.sql diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 4ec32883e6b..3f14209d141 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -1,7 +1,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -145,7 +147,34 @@ BlockInputStreams StorageBuffer::read( if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); - streams_from_dst = destination->read(column_names, query_info, context, processed_stage, max_block_size, num_streams); + const Block structure_of_destination_table = + allow_materialized ? destination->getSampleBlock() : destination->getSampleBlockNonMaterialized(); + + bool can_read_from_destination = true; + for (const String& column_name : column_names) + { + if (!structure_of_destination_table.has(column_name)) + { + LOG_WARNING(log, "Destination table " << destination_database << "." << destination_table + << " doesn't have column " << column_name << ". Data from destination table is skipped."); + can_read_from_destination = false; + break; + } + auto col = getColumn(column_name); + auto dst_col = structure_of_destination_table.getByName(column_name); + if (!dst_col.type->equals(*col.type)) + { + LOG_WARNING(log, "Destination table " << destination_database << "." << destination_table + << " have different type of column " << column_name << " (" + << col.type->getName() << " != " << dst_col.type->getName() + << "). Data from destination table is skipped."); + can_read_from_destination = false; + break; + } + } + + if (can_read_from_destination) + streams_from_dst = destination->read(column_names, query_info, context, processed_stage, max_block_size, num_streams); } BlockInputStreams streams_from_buffers; @@ -233,6 +262,9 @@ public: if (!block) return; + // Check table structure. + storage.check(block, true); + size_t rows = block.rows(); if (!rows) return; @@ -241,23 +273,8 @@ public: if (!storage.no_destination) { destination = storage.context.tryGetTable(storage.destination_database, storage.destination_table); - - if (destination) - { - if (destination.get() == &storage) - throw Exception("Destination table is myself. Write will cause infinite loop.", ErrorCodes::INFINITE_LOOP); - - /// Check table structure. - try - { - destination->check(block, true); - } - catch (Exception & e) - { - e.addMessage("(when looking at destination table " + storage.destination_database + "." + storage.destination_table + ")"); - throw; - } - } + if (destination.get() == &storage) + throw Exception("Destination table is myself. Write will cause infinite loop.", ErrorCodes::INFINITE_LOOP); } size_t bytes = block.bytes(); @@ -561,48 +578,46 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl * This will support some of the cases (but not all) when the table structure does not match. */ Block structure_of_destination_table = allow_materialized ? table->getSampleBlock() : table->getSampleBlockNonMaterialized(); - Names columns_intersection; - columns_intersection.reserve(block.columns()); + Block block_to_write; for (size_t i : ext::range(0, structure_of_destination_table.columns())) { auto dst_col = structure_of_destination_table.getByPosition(i); if (block.has(dst_col.name)) { - if (!block.getByName(dst_col.name).type->equals(*dst_col.type)) + auto column = block.getByName(dst_col.name); + if (!column.type->equals(*dst_col.type)) { - LOG_ERROR(log, "Destination table " << destination_database << "." << destination_table - << " have different type of column " << dst_col.name << " (" - << block.getByName(dst_col.name).type->getName() << " != " << dst_col.type->getName() - << "). Block of data is discarded."); - return; + LOG_WARNING(log, "Destination table " << destination_database << "." << destination_table + << " have different type of column " << column.name << " (" + << column.type->getName() << " != " << dst_col.type->getName() + << "). Block of data is converted."); + column.column = castColumn(column, dst_col.type, context); + column.type = dst_col.type; } - columns_intersection.push_back(dst_col.name); + block_to_write.insert(column); } } - if (columns_intersection.empty()) + if (block_to_write.columns() == 0) { - LOG_ERROR(log, "Destination table " << destination_database << "." << destination_table << " have no common columns with block in buffer. Block of data is discarded."); + LOG_ERROR(log, "Destination table " << destination_database << "." << destination_table + << " have no common columns with block in buffer. Block of data is discarded."); return; } - if (columns_intersection.size() != block.columns()) + if (block_to_write.columns() != block.columns()) LOG_WARNING(log, "Not all columns from block in buffer exist in destination table " << destination_database << "." << destination_table << ". Some columns are discarded."); auto list_of_columns = std::make_shared(); insert->columns = list_of_columns; - list_of_columns->children.reserve(columns_intersection.size()); - for (const String & column : columns_intersection) - list_of_columns->children.push_back(std::make_shared(column)); + list_of_columns->children.reserve(block_to_write.columns()); + for (const auto& column : block_to_write) + list_of_columns->children.push_back(std::make_shared(column.name)); InterpreterInsertQuery interpreter{insert, context, allow_materialized}; - Block block_to_write; - for (const auto & name : columns_intersection) - block_to_write.insert(block.getByName(name)); - auto block_io = interpreter.execute(); block_io.out->writePrefix(); block_io.out->write(block_to_write); diff --git a/dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.reference b/dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.reference new file mode 100644 index 00000000000..00eccfa771d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.reference @@ -0,0 +1,19 @@ +init +1 100 +2 200 +- +1 100 +2 200 +3 300 +alt +100 DEFZ +200 DEFZ +- +3 300 +4 400 +opt +100 DEFZ +200 DEFZ +300 DEFZ +400 DEFZ +- diff --git a/dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.sql b/dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.sql new file mode 100644 index 00000000000..5c348d6bc61 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.sql @@ -0,0 +1,32 @@ +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.buffer; +SET send_logs_level = 'error'; + +CREATE TABLE test.dst (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE test.buffer (x UInt64, y UInt64) ENGINE = Buffer(test, dst, 1, 99999, 99999, 1, 1, 99999, 99999); + +INSERT INTO test.buffer VALUES (1, 100); +INSERT INTO test.buffer VALUES (2, 200); +INSERT INTO test.buffer VALUES (3, 300); +SELECT 'init'; +SELECT * FROM test.dst ORDER BY x; +SELECT '-'; +SELECT * FROM test.buffer ORDER BY x; + +ALTER TABLE test.dst DROP COLUMN x, MODIFY COLUMN y String, ADD COLUMN z String DEFAULT 'DEFZ'; + +INSERT INTO test.buffer VALUES (4, 400); +SELECT 'alt'; +SELECT * FROM test.dst ORDER BY y; +SELECT '-'; +SELECT * FROM test.buffer ORDER BY x; + +OPTIMIZE TABLE test.buffer; +SELECT 'opt'; +SELECT * FROM test.dst ORDER BY y; +SELECT '-'; +SELECT * FROM test.buffer ORDER BY x; + +SET send_logs_level = 'warning'; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.buffer; From 33eb64269fa6c0e4a4787c720e94aaf4b4996579 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 19 Nov 2018 18:14:44 +0300 Subject: [PATCH 02/47] Update StorageBuffer.cpp --- dbms/src/Storages/StorageBuffer.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 3f14209d141..50f914edd4d 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -151,11 +151,11 @@ BlockInputStreams StorageBuffer::read( allow_materialized ? destination->getSampleBlock() : destination->getSampleBlockNonMaterialized(); bool can_read_from_destination = true; - for (const String& column_name : column_names) + for (const String & column_name : column_names) { if (!structure_of_destination_table.has(column_name)) { - LOG_WARNING(log, "Destination table " << destination_database << "." << destination_table + LOG_WARNING(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) << " doesn't have column " << column_name << ". Data from destination table is skipped."); can_read_from_destination = false; break; @@ -164,8 +164,8 @@ BlockInputStreams StorageBuffer::read( auto dst_col = structure_of_destination_table.getByName(column_name); if (!dst_col.type->equals(*col.type)) { - LOG_WARNING(log, "Destination table " << destination_database << "." << destination_table - << " have different type of column " << column_name << " (" + LOG_WARNING(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) + << " has different type of column " << backQuoteIfNeed(column_name) << " (" << col.type->getName() << " != " << dst_col.type->getName() << "). Data from destination table is skipped."); can_read_from_destination = false; From 9a3701cc5099a44e74e31a79462b43d64a388a73 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 19 Nov 2018 18:20:05 +0300 Subject: [PATCH 03/47] Update StorageBuffer.cpp --- dbms/src/Storages/StorageBuffer.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 50f914edd4d..4e2d58fc727 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -565,7 +565,7 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl if (!table) { - LOG_ERROR(log, "Destination table " << destination_database << "." << destination_table << " doesn't exist. Block of data is discarded."); + LOG_ERROR(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) << " doesn't exist. Block of data is discarded."); return; } @@ -587,8 +587,8 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl auto column = block.getByName(dst_col.name); if (!column.type->equals(*dst_col.type)) { - LOG_WARNING(log, "Destination table " << destination_database << "." << destination_table - << " have different type of column " << column.name << " (" + LOG_WARNING(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) + << " have different type of column " << backQuoteIfNeed(column.name) << " (" << column.type->getName() << " != " << dst_col.type->getName() << "). Block of data is converted."); column.column = castColumn(column, dst_col.type, context); @@ -601,14 +601,14 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl if (block_to_write.columns() == 0) { - LOG_ERROR(log, "Destination table " << destination_database << "." << destination_table - << " have no common columns with block in buffer. Block of data is discarded."); + LOG_ERROR(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) + << " have no common columns with block in buffer. Block of data is discarded."); return; } if (block_to_write.columns() != block.columns()) LOG_WARNING(log, "Not all columns from block in buffer exist in destination table " - << destination_database << "." << destination_table << ". Some columns are discarded."); + << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) << ". Some columns are discarded."); auto list_of_columns = std::make_shared(); insert->columns = list_of_columns; From 72fb78a0f851a2cb713f43a77a12907b0515b283 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 19 Nov 2018 18:20:34 +0300 Subject: [PATCH 04/47] Update StorageBuffer.cpp --- dbms/src/Storages/StorageBuffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 4e2d58fc727..460b5498840 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -613,7 +613,7 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl auto list_of_columns = std::make_shared(); insert->columns = list_of_columns; list_of_columns->children.reserve(block_to_write.columns()); - for (const auto& column : block_to_write) + for (const auto & column : block_to_write) list_of_columns->children.push_back(std::make_shared(column.name)); InterpreterInsertQuery interpreter{insert, context, allow_materialized}; From 01501fa8dbd909baf9c7da57d1c74a4c996b60db Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 23 Nov 2018 20:39:16 +0300 Subject: [PATCH 05/47] correct column list for rewritten INSERT query into Distributed [#CLICKHOUSE-4161] --- dbms/src/Storages/StorageDistributed.cpp | 34 +++++++++++------------- 1 file changed, 16 insertions(+), 18 deletions(-) diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 5d3860f449d..f65b33d3989 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include @@ -75,25 +76,22 @@ ASTPtr rewriteSelectQuery(const ASTPtr & query, const std::string & database, co return modified_query_ast; } -/// insert query has database and table names as bare strings -/// If the query is null, it creates a insert query with the database and tables -/// Or it creates a copy of query, changes the database and table names. -ASTPtr rewriteInsertQuery(const ASTPtr & query, const std::string & database, const std::string & table) +/// The columns list in the original INSERT query is incorrect because inserted blocks are transformed +/// to the form of the sample block of the Distributed table. So we rewrite it and add all columns from +/// the sample block instead. +ASTPtr createInsertToRemoteTableQuery(const std::string & database, const std::string & table, const Block & sample_block) { - ASTPtr modified_query_ast = nullptr; - if (query == nullptr) - modified_query_ast = std::make_shared(); - else - modified_query_ast = query->clone(); + auto query = std::make_shared(); + query->database = database; + query->table = table; - auto & actual_query = typeid_cast(*modified_query_ast); - actual_query.database = database; - actual_query.table = table; - actual_query.table_function = nullptr; - /// make sure query is not INSERT SELECT - actual_query.select = nullptr; + auto columns = std::make_shared(); + query->columns = columns; + query->children.push_back(columns); + for (const auto & col : sample_block) + columns->children.push_back(std::make_shared(col.name)); - return modified_query_ast; + return query; } /// Calculate maximum number in file names in directory and all subdirectories. @@ -274,7 +272,7 @@ BlockInputStreams StorageDistributed::read( } -BlockOutputStreamPtr StorageDistributed::write(const ASTPtr & query, const Settings & settings) +BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const Settings & settings) { auto cluster = getCluster(); @@ -298,7 +296,7 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr & query, const Setti /// DistributedBlockOutputStream will not own cluster, but will own ConnectionPools of the cluster return std::make_shared( - *this, rewriteInsertQuery(query, remote_database, remote_table), cluster, settings, insert_sync, timeout); + *this, createInsertToRemoteTableQuery(remote_database, remote_table, getSampleBlock()), cluster, settings, insert_sync, timeout); } From f09cb566b18bd6f0ab5a080a806cc5412f152b23 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Mon, 26 Nov 2018 17:23:53 +0300 Subject: [PATCH 06/47] add tests [#CLICKHOUSE-4161] --- .../test.py | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/dbms/tests/integration/test_insert_into_distributed_sync_async/test.py b/dbms/tests/integration/test_insert_into_distributed_sync_async/test.py index 552cb957c77..51f4b48a181 100755 --- a/dbms/tests/integration/test_insert_into_distributed_sync_async/test.py +++ b/dbms/tests/integration/test_insert_into_distributed_sync_async/test.py @@ -49,6 +49,28 @@ def test_insertion_sync(started_cluster): assert node2.query("SELECT count() FROM local_table").rstrip() == '20000' + # Insert with explicitly specified columns. + node1.query(''' + SET insert_distributed_sync = 1, insert_distributed_timeout = 1; + INSERT INTO distributed_table(date, val) VALUES ('2000-01-01', 100500)''') + + # Insert with columns specified in different order. + node1.query(''' + SET insert_distributed_sync = 1, insert_distributed_timeout = 1; + INSERT INTO distributed_table(val, date) VALUES (100500, '2000-01-01')''') + + # Insert with an incomplete list of columns. + node1.query(''' + SET insert_distributed_sync = 1, insert_distributed_timeout = 1; + INSERT INTO distributed_table(val) VALUES (100500)''') + + expected = TSV(''' +0000-00-00 100500 +2000-01-01 100500 +2000-01-01 100500''') + assert TSV(node2.query('SELECT date, val FROM local_table WHERE val = 100500 ORDER BY date')) == expected + + """ def test_insertion_sync_fails_on_error(started_cluster): with PartitionManager() as pm: From f38a320909f8de82170c4c22e7b349a51daa336f Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Mon, 26 Nov 2018 17:52:43 +0300 Subject: [PATCH 07/47] fix insert batching test (the INSERT queries are rewritten differently now) [#CLICKHOUSE-4161] --- .../integration/test_insert_into_distributed/test.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/dbms/tests/integration/test_insert_into_distributed/test.py b/dbms/tests/integration/test_insert_into_distributed/test.py index 9da48ec55ee..d6700106929 100644 --- a/dbms/tests/integration/test_insert_into_distributed/test.py +++ b/dbms/tests/integration/test_insert_into_distributed/test.py @@ -120,16 +120,16 @@ def test_inserts_batching(started_cluster): # Batches of max 3 rows are formed as min_insert_block_size_rows = 3. # Blocks: # 1. Failed batch that is retried with the same contents. - # 2. Full batch of inserts with (d, x) order of columns. - # 3. Full batch of inserts with (x, d) order of columns. + # 2. Full batch of inserts before ALTER. + # 3. Full batch of inserts before ALTER. # 4. Full batch of inserts after ALTER (that have different block structure). - # 5. What was left to insert with (d, x) order before ALTER. + # 5. What was left to insert with the column structure before ALTER. expected = '''\ 20000101_20000101_1_1_0\t[1] -20000101_20000101_2_2_0\t[3,4,5] -20000101_20000101_3_3_0\t[2,7,8] +20000101_20000101_2_2_0\t[2,3,4] +20000101_20000101_3_3_0\t[5,6,7] 20000101_20000101_4_4_0\t[10,11,12] -20000101_20000101_5_5_0\t[6,9] +20000101_20000101_5_5_0\t[8,9] ''' assert TSV(result) == TSV(expected) From 44ca56c731069d8a4ed36f45c39bea469055ada7 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 27 Nov 2018 03:43:58 +0300 Subject: [PATCH 08/47] CLICKHOUSE-4127: Convert destination table's data when reading from StorageBuffer. --- .../AddingDefaultBlockInputStream.cpp | 28 +++++++ .../AddingDefaultBlockInputStream.h | 40 ++++++++++ .../AddingDefaultBlockOutputStream.cpp | 71 +--------------- dbms/src/Interpreters/addMissingDefaults.cpp | 80 +++++++++++++++++++ dbms/src/Interpreters/addMissingDefaults.h | 26 ++++++ dbms/src/Storages/StorageBuffer.cpp | 67 ++++++++++------ ...r_destination_for_storage_buffer.reference | 6 ++ ...3_alter_destination_for_storage_buffer.sql | 4 +- 8 files changed, 226 insertions(+), 96 deletions(-) create mode 100644 dbms/src/DataStreams/AddingDefaultBlockInputStream.cpp create mode 100644 dbms/src/DataStreams/AddingDefaultBlockInputStream.h create mode 100644 dbms/src/Interpreters/addMissingDefaults.cpp create mode 100644 dbms/src/Interpreters/addMissingDefaults.h diff --git a/dbms/src/DataStreams/AddingDefaultBlockInputStream.cpp b/dbms/src/DataStreams/AddingDefaultBlockInputStream.cpp new file mode 100644 index 00000000000..749eebda1a5 --- /dev/null +++ b/dbms/src/DataStreams/AddingDefaultBlockInputStream.cpp @@ -0,0 +1,28 @@ +#include +#include + + +namespace DB +{ + +AddingDefaultBlockInputStream::AddingDefaultBlockInputStream( + const BlockInputStreamPtr & input_, + const Block & header_, + const ColumnDefaults & column_defaults_, + const Context & context_) + : input(input_), header(header_), + column_defaults(column_defaults_), context(context_) +{ + children.emplace_back(input); +} + +Block AddingDefaultBlockInputStream::readImpl() +{ + Block src = children.back()->read(); + if (!src) + return src; + + return addMissingDefaults(src, header.getNamesAndTypesList(), column_defaults, context); +} + +} diff --git a/dbms/src/DataStreams/AddingDefaultBlockInputStream.h b/dbms/src/DataStreams/AddingDefaultBlockInputStream.h new file mode 100644 index 00000000000..c0afffbfc17 --- /dev/null +++ b/dbms/src/DataStreams/AddingDefaultBlockInputStream.h @@ -0,0 +1,40 @@ +#pragma once + +#include +#include + + +namespace DB +{ + + +/** This stream adds three types of columns into block + * 1. Columns, that are missed inside request, but present in table without defaults (missed columns) + * 2. Columns, that are missed inside request, but present in table with defaults (columns with default values) + * 3. Columns that materialized from other columns (materialized columns) + * All three types of columns are materialized (not constants). + */ +class AddingDefaultBlockInputStream : public IProfilingBlockInputStream +{ +public: + AddingDefaultBlockInputStream( + const BlockInputStreamPtr & input_, + const Block & header_, + const ColumnDefaults & column_defaults_, + const Context & context_); + + String getName() const override { return "AddingDefault"; } + Block getHeader() const override { return header; } + +private: + Block readImpl() override; + + BlockInputStreamPtr input; + /// Blocks after this stream should have this structure + const Block header; + const ColumnDefaults column_defaults; + const Context & context; +}; + + +} diff --git a/dbms/src/DataStreams/AddingDefaultBlockOutputStream.cpp b/dbms/src/DataStreams/AddingDefaultBlockOutputStream.cpp index 4b3f216e182..a8f78c434e7 100644 --- a/dbms/src/DataStreams/AddingDefaultBlockOutputStream.cpp +++ b/dbms/src/DataStreams/AddingDefaultBlockOutputStream.cpp @@ -1,11 +1,5 @@ #include - -#include -#include -#include -#include -#include -#include +#include namespace DB @@ -13,68 +7,7 @@ namespace DB void AddingDefaultBlockOutputStream::write(const Block & block) { - Block res; - /// We take given columns from input block - /// and missed columns without default value (default and meterialized will be computed later) - for (const auto & column : output_block) - { - if (block.has(column.name)) - res.insert(block.getByName(column.name)); - else if (!column_defaults.count(column.name)) - res.insert(column); - } - - /// Adds not specified default values. - size_t rows = block.rows(); - - /// For missing columns of nested structure, you need to create not a column of empty arrays, but a column of arrays of correct lengths. - /// First, remember the offset columns for all arrays in the block. - std::map offset_columns; - - for (size_t i = 0, size = block.columns(); i < size; ++i) - { - const auto & elem = block.getByPosition(i); - - if (const ColumnArray * array = typeid_cast(&*elem.column)) - { - String offsets_name = Nested::extractTableName(elem.name); - auto & offsets_column = offset_columns[offsets_name]; - - /// If for some reason there are different offset columns for one nested structure, then we take nonempty. - if (!offsets_column || offsets_column->empty()) - offsets_column = array->getOffsetsPtr(); - } - } - - /// In this loop we fill missed columns - for (auto & column : res) - { - if (block.has(column.name)) - continue; - - String offsets_name = Nested::extractTableName(column.name); - if (offset_columns.count(offsets_name)) - { - ColumnPtr offsets_column = offset_columns[offsets_name]; - DataTypePtr nested_type = typeid_cast(*column.type).getNestedType(); - UInt64 nested_rows = rows ? get((*offsets_column)[rows - 1]) : 0; - - ColumnPtr nested_column = nested_type->createColumnConstWithDefaultValue(nested_rows)->convertToFullColumnIfConst(); - column.column = ColumnArray::create(nested_column, offsets_column); - } - else - { - /** It is necessary to turn a constant column into a full column, since in part of blocks (from other parts), - * it can be full (or the interpreter may decide that it is constant everywhere). - */ - column.column = column.type->createColumnConstWithDefaultValue(rows)->convertToFullColumnIfConst(); - } - } - - /// Computes explicitly specified values (in column_defaults) by default and materialized columns. - evaluateMissingDefaults(res, output_block.getNamesAndTypesList(), column_defaults, context); - - output->write(res); + output->write(addMissingDefaults(block, output_block.getNamesAndTypesList(), column_defaults, context)); } void AddingDefaultBlockOutputStream::flush() diff --git a/dbms/src/Interpreters/addMissingDefaults.cpp b/dbms/src/Interpreters/addMissingDefaults.cpp new file mode 100644 index 00000000000..10318ee89cf --- /dev/null +++ b/dbms/src/Interpreters/addMissingDefaults.cpp @@ -0,0 +1,80 @@ +#include + +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +Block addMissingDefaults(const Block & block, + const NamesAndTypesList & required_columns, + const ColumnDefaults & column_defaults, + const Context & context) +{ + /// For missing columns of nested structure, you need to create not a column of empty arrays, but a column of arrays of correct lengths. + /// First, remember the offset columns for all arrays in the block. + std::map offset_columns; + + for (size_t i = 0, size = block.columns(); i < size; ++i) + { + const auto & elem = block.getByPosition(i); + + if (const ColumnArray * array = typeid_cast(&*elem.column)) + { + String offsets_name = Nested::extractTableName(elem.name); + auto & offsets_column = offset_columns[offsets_name]; + + /// If for some reason there are different offset columns for one nested structure, then we take nonempty. + if (!offsets_column || offsets_column->empty()) + offsets_column = array->getOffsetsPtr(); + } + } + + const size_t rows = block.rows(); + Block res; + + /// We take given columns from input block and missed columns without default value + /// (default and materialized will be computed later). + for (const auto & column : required_columns) + { + if (block.has(column.name)) + { + res.insert(block.getByName(column.name)); + continue; + } + + if (column_defaults.count(column.name)) + continue; + + String offsets_name = Nested::extractTableName(column.name); + if (offset_columns.count(offsets_name)) + { + ColumnPtr offsets_column = offset_columns[offsets_name]; + DataTypePtr nested_type = typeid_cast(*column.type).getNestedType(); + UInt64 nested_rows = rows ? get((*offsets_column)[rows - 1]) : 0; + + ColumnPtr nested_column = nested_type->createColumnConstWithDefaultValue(nested_rows)->convertToFullColumnIfConst(); + auto new_column = ColumnArray::create(nested_column, offsets_column); + res.insert(ColumnWithTypeAndName(std::move(new_column), column.type, column.name)); + continue; + } + + /** It is necessary to turn a constant column into a full column, since in part of blocks (from other parts), + * it can be full (or the interpreter may decide that it is constant everywhere). + */ + auto new_column = column.type->createColumnConstWithDefaultValue(rows)->convertToFullColumnIfConst(); + res.insert(ColumnWithTypeAndName(std::move(new_column), column.type, column.name)); + } + + /// Computes explicitly specified values (in column_defaults) by default and materialized columns. + evaluateMissingDefaults(res, required_columns, column_defaults, context); + return res; +} + +} diff --git a/dbms/src/Interpreters/addMissingDefaults.h b/dbms/src/Interpreters/addMissingDefaults.h new file mode 100644 index 00000000000..d9d8d3d4f22 --- /dev/null +++ b/dbms/src/Interpreters/addMissingDefaults.h @@ -0,0 +1,26 @@ +#pragma once + +#include + + +namespace DB +{ + +class Block; +class Context; +class NamesAndTypesList; +struct ColumnDefault; + +/** Adds three types of columns into block + * 1. Columns, that are missed inside request, but present in table without defaults (missed columns) + * 2. Columns, that are missed inside request, but present in table with defaults (columns with default values) + * 3. Columns that materialized from other columns (materialized columns) + * All three types of columns are materialized (not constants). + */ +Block addMissingDefaults( + const Block & block, + const NamesAndTypesList & required_columns, + const std::unordered_map & column_defaults, + const Context & context); + +} diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 460b5498840..6f850a6df30 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -147,34 +148,50 @@ BlockInputStreams StorageBuffer::read( if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); - const Block structure_of_destination_table = - allow_materialized ? destination->getSampleBlock() : destination->getSampleBlockNonMaterialized(); - - bool can_read_from_destination = true; + /// Collect columns from the destination tables which can be requested. + /// Find out if there is a struct mismatch and we need to convert read blocks from the destination tables. + Names columns_intersection; + bool struct_mismatch = false; for (const String & column_name : column_names) { - if (!structure_of_destination_table.has(column_name)) - { - LOG_WARNING(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) - << " doesn't have column " << column_name << ". Data from destination table is skipped."); - can_read_from_destination = false; - break; - } - auto col = getColumn(column_name); - auto dst_col = structure_of_destination_table.getByName(column_name); - if (!dst_col.type->equals(*col.type)) - { - LOG_WARNING(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) - << " has different type of column " << backQuoteIfNeed(column_name) << " (" - << col.type->getName() << " != " << dst_col.type->getName() - << "). Data from destination table is skipped."); - can_read_from_destination = false; - break; - } + if (destination->hasColumn(column_name)) + { + columns_intersection.emplace_back(column_name); + if (!destination->getColumn(column_name).type->equals(*getColumn(column_name).type)) + { + LOG_WARNING(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) + << " has different type of column " << backQuoteIfNeed(column_name) << " (" + << destination->getColumn(column_name).type->getName() << " != " << getColumn(column_name).type->getName() + << "). Data from destination table is converted."); + struct_mismatch = true; + } + } + else + { + LOG_WARNING(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) + << " doesn't have column " << backQuoteIfNeed(column_name) << ". The default values are used."); + struct_mismatch = true; + } } - if (can_read_from_destination) - streams_from_dst = destination->read(column_names, query_info, context, processed_stage, max_block_size, num_streams); + if (columns_intersection.empty()) + LOG_WARNING(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) + << " has no common columns with block in buffer. Block of data is skipped."); + else + streams_from_dst = destination->read(columns_intersection, query_info, context, processed_stage, max_block_size, num_streams); + + if (struct_mismatch && !streams_from_dst.empty()) + { + /// Add streams to convert read blocks from the destination table. + auto header = getSampleBlock(); + for (auto& stream_from_dst : streams_from_dst) + { + stream_from_dst = std::make_shared( + stream_from_dst, header, getColumns().defaults, context); + stream_from_dst = std::make_shared( + context, stream_from_dst, header, ConvertingBlockInputStream::MatchColumnsMode::Name); + } + } } BlockInputStreams streams_from_buffers; @@ -589,7 +606,7 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl { LOG_WARNING(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) << " have different type of column " << backQuoteIfNeed(column.name) << " (" - << column.type->getName() << " != " << dst_col.type->getName() + << dst_col.type->getName() << " != " << column.type->getName() << "). Block of data is converted."); column.column = castColumn(column, dst_col.type, context); column.type = dst_col.type; diff --git a/dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.reference b/dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.reference index 00eccfa771d..c4e3ee1f88e 100644 --- a/dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.reference +++ b/dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.reference @@ -9,6 +9,8 @@ alt 100 DEFZ 200 DEFZ - +0 100 +0 200 3 300 4 400 opt @@ -17,3 +19,7 @@ opt 300 DEFZ 400 DEFZ - +0 100 +0 200 +0 300 +0 400 diff --git a/dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.sql b/dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.sql index 5c348d6bc61..2193f5984b9 100644 --- a/dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.sql +++ b/dbms/tests/queries/0_stateless/00753_alter_destination_for_storage_buffer.sql @@ -19,13 +19,13 @@ INSERT INTO test.buffer VALUES (4, 400); SELECT 'alt'; SELECT * FROM test.dst ORDER BY y; SELECT '-'; -SELECT * FROM test.buffer ORDER BY x; +SELECT * FROM test.buffer ORDER BY y; OPTIMIZE TABLE test.buffer; SELECT 'opt'; SELECT * FROM test.dst ORDER BY y; SELECT '-'; -SELECT * FROM test.buffer ORDER BY x; +SELECT * FROM test.buffer ORDER BY y; SET send_logs_level = 'warning'; DROP TABLE IF EXISTS test.dst; From 3a6d5ad4890e5f49392bf06a52df9b8c4c35e80d Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 27 Nov 2018 14:13:59 +0300 Subject: [PATCH 09/47] WIP on website+docs (#3668) * CLICKHOUSE-4063: less manual html @ index.md * CLICKHOUSE-4063: recommend markdown="1" in README.md * CLICKHOUSE-4003: manually purge custom.css for now * CLICKHOUSE-4064: expand
before any print (including to pdf) * CLICKHOUSE-3927: rearrange interfaces/formats.md a bit * CLICKHOUSE-3306: add few http headers * Remove copy-paste introduced in #3392 * Hopefully better chinese fonts #3392 * get rid of tabs @ custom.css * Apply comments and patch from #3384 * Add jdbc.md to ToC and some translation, though it still looks badly incomplete * minor punctuation * Add some backlinks to official website from mirrors that just blindly take markdown sources * Do not make fonts extra light * find . -name '*.md' -type f | xargs -I{} perl -pi -e 's//g' {} * find . -name '*.md' -type f | xargs -I{} perl -pi -e 's/ sql/g' {} * Remove outdated stuff from roadmap.md * Not so light font on front page too * Refactor Chinese formats.md to match recent changes in other languages * Update some links on front page * Remove some outdated comment * Add twitter link to front page * More front page links tuning * Add Amsterdam meetup link * Smaller font to avoid second line * Add Amsterdam link to README.md * Proper docs nav translation * Back to 300 font-weight except Chinese * fix docs build * Update Amsterdam link * remove symlinks * more zh punctuation * apply lost comment by @zhang2014 * Apply comments by @zhang2014 from #3417 * Remove Beijing link * rm incorrect symlink * restore content of docs/zh/operations/table_engines/index.md * CLICKHOUSE-3751: stem terms while searching docs * CLICKHOUSE-3751: use English stemmer in non-English docs too * CLICKHOUSE-4135 fix * Remove past meetup link * Add blog link to top nav * Add ContentSquare article link * Add form link to front page + refactor some texts * couple markup fixes * minor * Introduce basic ODBC driver page in docs * More verbose 3rd party libs disclaimer * Put third-party stuff into a separate folder * Separate third-party stuff in ToC too * Update links * Move stuff that is not really (only) a client library into a separate page * Add clickhouse-hdfs-loader link * Some introduction for "interfaces" section * Rewrite tcp.md * http_interface.md -> http.md * fix link * Remove unconvenient error for now * try to guess anchor instead of failing * remove symlink * Remove outdated info from introduction * remove ru roadmap.md * replace ru roadmap.md with symlink * Update roadmap.md * lost file * Title case in toc_en.yml * Sync "Functions" ToC section with en * Remove reference to pretty old ClickHouse release from docs * couple lost symlinks in fa --- .../interfaces/{http_interface.md => http.md} | 0 docs/en/interfaces/index.md | 14 +- docs/en/interfaces/jdbc.md | 2 +- docs/en/interfaces/odbc.md | 6 + docs/en/interfaces/tcp.md | 3 +- .../client_libraries.md} | 20 +-- .../gui.md} | 0 .../en/interfaces/third-party/integrations.md | 25 +++ .../settings/permissions_for_queries.md | 2 +- .../table_engines/custom_partitioning_key.md | 2 - docs/en/roadmap.md | 16 +- .../interfaces/{http_interface.md => http.md} | 0 docs/fa/interfaces/index.md | 17 +- docs/fa/interfaces/jdbc.md | 2 - docs/fa/interfaces/odbc.md | 7 + docs/fa/interfaces/tcp.md | 4 +- .../client_libraries.md} | 20 ++- .../gui.md} | 0 .../fa/interfaces/third-party/integrations.md | 27 +++ .../features_considered_disadvantages.md | 2 +- .../functions/functions_for_nulls.md | 1 + docs/fa/query_language/functions/geo.md | 1 + docs/redirects.txt | 3 + .../interfaces/{http_interface.md => http.md} | 0 docs/ru/interfaces/index.md | 15 +- docs/ru/interfaces/odbc.md | 5 + docs/ru/interfaces/tcp.md | 2 +- .../client_libraries.md} | 20 +-- .../gui.md} | 0 .../ru/interfaces/third-party/integrations.md | 24 +++ .../features_considered_disadvantages.md | 2 +- .../table_engines/custom_partitioning_key.md | 2 - docs/ru/roadmap.md | 15 +- docs/toc_en.yml | 169 +++++++++--------- docs/toc_fa.yml | 65 +++---- docs/toc_ru.yml | 16 +- docs/toc_zh.yml | 67 +++---- docs/tools/concatenate.py | 8 +- .../interfaces/{http_interface.md => http.md} | 0 docs/zh/interfaces/index.md | 13 +- docs/zh/interfaces/odbc.md | 5 + docs/zh/interfaces/tcp.md | 3 +- .../client_libraries.md} | 16 +- .../gui.md} | 0 .../zh/interfaces/third-party/integrations.md | 25 +++ .../features_considered_disadvantages.md | 8 +- website/index.html | 20 +-- 47 files changed, 429 insertions(+), 245 deletions(-) rename docs/en/interfaces/{http_interface.md => http.md} (100%) create mode 100644 docs/en/interfaces/odbc.md rename docs/en/interfaces/{third-party_client_libraries.md => third-party/client_libraries.md} (84%) rename docs/en/interfaces/{third-party_gui.md => third-party/gui.md} (100%) create mode 100644 docs/en/interfaces/third-party/integrations.md rename docs/fa/interfaces/{http_interface.md => http.md} (100%) create mode 100644 docs/fa/interfaces/odbc.md rename docs/fa/interfaces/{third-party_client_libraries.md => third-party/client_libraries.md} (81%) rename docs/fa/interfaces/{third-party_gui.md => third-party/gui.md} (100%) create mode 100644 docs/fa/interfaces/third-party/integrations.md create mode 120000 docs/fa/query_language/functions/functions_for_nulls.md create mode 120000 docs/fa/query_language/functions/geo.md rename docs/ru/interfaces/{http_interface.md => http.md} (100%) create mode 100644 docs/ru/interfaces/odbc.md rename docs/ru/interfaces/{third-party_client_libraries.md => third-party/client_libraries.md} (81%) rename docs/ru/interfaces/{third-party_gui.md => third-party/gui.md} (100%) create mode 100644 docs/ru/interfaces/third-party/integrations.md mode change 100644 => 120000 docs/ru/roadmap.md rename docs/zh/interfaces/{http_interface.md => http.md} (100%) create mode 100644 docs/zh/interfaces/odbc.md rename docs/zh/interfaces/{third-party_client_libraries.md => third-party/client_libraries.md} (85%) rename docs/zh/interfaces/{third-party_gui.md => third-party/gui.md} (100%) create mode 100644 docs/zh/interfaces/third-party/integrations.md mode change 120000 => 100644 docs/zh/introduction/features_considered_disadvantages.md diff --git a/docs/en/interfaces/http_interface.md b/docs/en/interfaces/http.md similarity index 100% rename from docs/en/interfaces/http_interface.md rename to docs/en/interfaces/http.md diff --git a/docs/en/interfaces/index.md b/docs/en/interfaces/index.md index 9f445d45229..e11e1a16ebc 100644 --- a/docs/en/interfaces/index.md +++ b/docs/en/interfaces/index.md @@ -2,7 +2,19 @@ # Interfaces -To explore the system's capabilities, download data to tables, or make manual queries, use the clickhouse-client program. +ClickHouse provides two network interfaces (both can be optionally wrapped in TLS for additional security): +* [HTTP](http.md), which is documented and easy to use directly. +* [Native TCP](tcp.md), which has less overhead. + +In most cases it is recommended to use appropriate tool or library instead of interacting with those directly. Officially supported by Yandex are the following: +* [Command-line client](cli.md) +* [JDBC driver](jdbc.md) +* [ODBC driver](odbc.md) + +There are also a wide range of third-party libraries for working with ClickHouse: +* [Client libraries](third-party/client_libraries.md) +* [Integrations](third-party/integrations.md) +* [Visual interfaces](third-party/gui.md) [Original article](https://clickhouse.yandex/docs/en/interfaces/) diff --git a/docs/en/interfaces/jdbc.md b/docs/en/interfaces/jdbc.md index 8454881dfb7..de735abfdfd 100644 --- a/docs/en/interfaces/jdbc.md +++ b/docs/en/interfaces/jdbc.md @@ -1,7 +1,7 @@ # JDBC Driver - [Official driver](https://github.com/yandex/clickhouse-jdbc). -- Third-party driver from [ClickHouse-Native-JDBC](https://github.com/housepower/ClickHouse-Native-JDBC). +- Third-party driver [ClickHouse-Native-JDBC](https://github.com/housepower/ClickHouse-Native-JDBC). [Original article](https://clickhouse.yandex/docs/en/interfaces/jdbc/) diff --git a/docs/en/interfaces/odbc.md b/docs/en/interfaces/odbc.md new file mode 100644 index 00000000000..845c4e21fb3 --- /dev/null +++ b/docs/en/interfaces/odbc.md @@ -0,0 +1,6 @@ +# ODBC Driver + +- [Official driver](https://github.com/yandex/clickhouse-odbc). + + +[Original article](https://clickhouse.yandex/docs/en/interfaces/odbc/) diff --git a/docs/en/interfaces/tcp.md b/docs/en/interfaces/tcp.md index 86ef118e882..c17e8c15b5e 100644 --- a/docs/en/interfaces/tcp.md +++ b/docs/en/interfaces/tcp.md @@ -1,6 +1,5 @@ # Native Interface (TCP) -The native interface is used in the "clickhouse-client" command-line client for interaction between servers with distributed query processing, and also in C++ programs. We will only cover the command-line client. - +The native protocol is used in the [command-line client](cli.md), for interserver communication during distributed query processing, and also in other C++ programs. Unfortunately, native ClickHouse protocol does not have formal specification yet, but it can be reverse engineered from ClickHouse source code (starting [around here](https://github.com/yandex/ClickHouse/tree/master/dbms/src/Client)) and/or by intercepting and analyzing TCP traffic. [Original article](https://clickhouse.yandex/docs/en/interfaces/tcp/) diff --git a/docs/en/interfaces/third-party_client_libraries.md b/docs/en/interfaces/third-party/client_libraries.md similarity index 84% rename from docs/en/interfaces/third-party_client_libraries.md rename to docs/en/interfaces/third-party/client_libraries.md index 3ae40e829dc..5878d6e0200 100644 --- a/docs/en/interfaces/third-party_client_libraries.md +++ b/docs/en/interfaces/third-party/client_libraries.md @@ -1,10 +1,10 @@ -# Libraries from Third-party Developers +# Client Libraries from Third-party Developers -We have not tested the libraries listed below. +!!! warning "Disclaimer" + Yandex does **not** maintain the libraries listed below and haven't done any extensive testing to ensure their quality. - Python - [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm) - - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) - [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-client](https://github.com/yurial/clickhouse-client) - [aiochclient](https://github.com/maximdanilchenko/aiochclient) @@ -30,23 +30,21 @@ We have not tested the libraries listed below. - R - [clickhouse-r](https://github.com/hannesmuehleisen/clickhouse-r) - [RClickhouse](https://github.com/IMSMWU/RClickhouse) +- Java + - [clickhouse-client-java](https://github.com/VirtusAI/clickhouse-client-java) - Scala - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) -- .NET +- Kotlin + - [AORM](https://github.com/TanVD/AORM) +- C# - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) - - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) - C++ - [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp/) - Elixir - [clickhousex](https://github.com/appodeal/clickhousex/) - - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) -- Java - - [clickhouse-client-java](https://github.com/VirtusAI/clickhouse-client-java) -- Kotlin - - [AORM](https://github.com/TanVD/AORM) - Nim - [nim-clickhouse](https://github.com/leonardoce/nim-clickhouse) -[Original article](https://clickhouse.yandex/docs/en/interfaces/third-party_client_libraries/) +[Original article](https://clickhouse.yandex/docs/en/interfaces/third-party/client_libraries/) diff --git a/docs/en/interfaces/third-party_gui.md b/docs/en/interfaces/third-party/gui.md similarity index 100% rename from docs/en/interfaces/third-party_gui.md rename to docs/en/interfaces/third-party/gui.md diff --git a/docs/en/interfaces/third-party/integrations.md b/docs/en/interfaces/third-party/integrations.md new file mode 100644 index 00000000000..ece64d9dcb3 --- /dev/null +++ b/docs/en/interfaces/third-party/integrations.md @@ -0,0 +1,25 @@ +# Integration Libraries from Third-party Developers + +!!! warning "Disclaimer" + Yandex does **not** maintain the libraries listed below and haven't done any extensive testing to ensure their quality. + +- Python + - [SQLAlchemy](https://www.sqlalchemy.org) + - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) +- Java + - [Hadoop](http://hadoop.apache.org) + - [clickhouse-hdfs-loader](https://github.com/jaykelin/clickhouse-hdfs-loader) (uses [JDBC](../jdbc.md)) +- Scala + - [Akka](https://akka.io) + - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) +- C# + - [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview) + - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) + - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) + - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) +- Elixir + - [Ecto](https://github.com/elixir-ecto/ecto) + - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) + + +[Original article](https://clickhouse.yandex/docs/en/interfaces/third-party/integrations/) \ No newline at end of file diff --git a/docs/en/operations/settings/permissions_for_queries.md b/docs/en/operations/settings/permissions_for_queries.md index 8c731ed1ce8..c65b35e60cc 100644 --- a/docs/en/operations/settings/permissions_for_queries.md +++ b/docs/en/operations/settings/permissions_for_queries.md @@ -33,7 +33,7 @@ See [above](#permissions_for_queries) for the division of queries into groups. After setting `readonly = 1`, a user can't change `readonly` and `allow_ddl` settings in the current session. -When using the `GET` method in the [HTTP interface](../../interfaces/http_interface.md#http_interface), `readonly = 1` is set automatically. To modify data use the `POST` method. +When using the `GET` method in the [HTTP interface](../../interfaces/http.md#http_interface), `readonly = 1` is set automatically. To modify data use the `POST` method. diff --git a/docs/en/operations/table_engines/custom_partitioning_key.md b/docs/en/operations/table_engines/custom_partitioning_key.md index 55940db8ca9..5bf686ea872 100644 --- a/docs/en/operations/table_engines/custom_partitioning_key.md +++ b/docs/en/operations/table_engines/custom_partitioning_key.md @@ -2,8 +2,6 @@ # Custom Partitioning Key -Starting with version 1.1.54310, you can create tables in the MergeTree family with any partitioning expression (not only partitioning by month). - The partition key can be an expression from the table columns, or a tuple of such expressions (similar to the primary key). The partition key can be omitted. When creating a table, specify the partition key in the ENGINE description with the new syntax: ``` diff --git a/docs/en/roadmap.md b/docs/en/roadmap.md index 490e570b659..46931d5983b 100644 --- a/docs/en/roadmap.md +++ b/docs/en/roadmap.md @@ -4,11 +4,23 @@ - JOIN syntax compatible with SQL standard: - Mutliple `JOIN`s in single `SELECT` +- Protobuf and Parquet input and output formats + +## Q1 2019 + +- Import/export from HDFS and S3 +- Lower metadata size in ZooKeeper +- Adaptive index granularity for MergeTree engine family + +## Q2 2019 - JOIN execution improvements: - Distributed join not limited by memory - -- Protobuf and Parquet input and output formats - Resource pools for more precise distribution of cluster capacity between users +## Q3 2019 + +- Fine-grained authorization +- Integration with external authentication services + [Original article](https://clickhouse.yandex/docs/en/roadmap/) diff --git a/docs/fa/interfaces/http_interface.md b/docs/fa/interfaces/http.md similarity index 100% rename from docs/fa/interfaces/http_interface.md rename to docs/fa/interfaces/http.md diff --git a/docs/fa/interfaces/index.md b/docs/fa/interfaces/index.md index d06c8ec8fd0..0c8914939a6 100644 --- a/docs/fa/interfaces/index.md +++ b/docs/fa/interfaces/index.md @@ -2,9 +2,22 @@
-# Interface ها +# رابط ها -برای کشف قابلیت های سیستم، دانلو داده ها به جداول، یا ساخت query های دستی، از برنامه clikhouse-client استفاده کنید. +ClickHouse دو اینترفیس شبکه را فراهم می کند (هر دو می توانند به صورت اختیاری در TLS برای امنیت اضافی پیچیده شوند): + +* [HTTP](http.md), که مستند شده و به راحتی به طور مستقیم استفاده می شود. +* [بومی TCP](tcp.md), که دارای سربار کمتر است. + +اگرچه در بیشتر موارد توصیه می شود از ابزار یا کتابخانه مناسب استفاده کنید تا به طور مستقیم با آن ها ارتباط برقرار نکنید. به طور رسمی توسط یانداکس پشتیبانی می شوند عبارتند از: +* [خط فرمان خط](cli.md) +* [راننده JDBC](jdbc.md) +* [راننده ODBC](odbc.md) + +همچنین برای کار با ClickHouse طیف گسترده ای از کتابخانه های شخص ثالث وجود دارد: +* [کتابخانه های مشتری](third-party/client_libraries.md) +* [ادغام](third-party/integrations.md) +* [رابط های بصری](third-party/gui.md)
[مقاله اصلی](https://clickhouse.yandex/docs/fa/interfaces/) diff --git a/docs/fa/interfaces/jdbc.md b/docs/fa/interfaces/jdbc.md index e72b5504c06..5cb82ff4274 100644 --- a/docs/fa/interfaces/jdbc.md +++ b/docs/fa/interfaces/jdbc.md @@ -5,10 +5,8 @@ درایور رسمی JDBC برای ClickHouse وجود دارد. برای اطلاعات بیشتر [اینجا](https://github.com/yandex/clickhouse-jdbc) را ببینید. -JDBC drivers implemented by other organizations: درایور JDBC توسط سازمان های دیگر اجرا می شوند. - - [ClickHouse-Native-JDBC](https://github.com/housepower/ClickHouse-Native-JDBC) diff --git a/docs/fa/interfaces/odbc.md b/docs/fa/interfaces/odbc.md new file mode 100644 index 00000000000..692fc92651d --- /dev/null +++ b/docs/fa/interfaces/odbc.md @@ -0,0 +1,7 @@ +
+# ODBC درایور + +درایور رسمی ODBC برای ClickHouse وجود دارد. برای اطلاعات بیشتر [اینجا](https://github.com/yandex/clickhouse-odbc) را ببینید. + +
+[مقاله اصلی](https://clickhouse.yandex/docs/fa/interfaces/odbc/) diff --git a/docs/fa/interfaces/tcp.md b/docs/fa/interfaces/tcp.md index 5e767b876f8..bd902aedc58 100644 --- a/docs/fa/interfaces/tcp.md +++ b/docs/fa/interfaces/tcp.md @@ -1,8 +1,8 @@
-# Native interface (TCP) +# رابط بومی (TCP) -native interface در محیط ترمینال "clickhouse-client" برای تعامل بین سرور با پردازش query توزیع شده مورد استفاده قرار می گیرد. همچنین native interface در برنامه های C++ مورد استفاده قرار می گیرد. ما فقط کلاینت command-line را پوشش میدیم. +پروتکل بومی در [خط فرمان خط] (cli.md)، برای برقراری ارتباط بین سرور در طی پردازش پرس و جو توزیع شده، و همچنین در سایر برنامه های C ++ استفاده می شود. متاسفانه، پروتکل ClickHouse بومی هنوز مشخصات رسمی ندارد، اما می توان آن را از کد منبع ClickHouse (شروع [از اینجا](https://github.com/yandex/ClickHouse/tree/master/dbms/src/Client)) و / یا با رهگیری و تجزیه و تحلیل ترافیک TCP.
[مقاله اصلی](https://clickhouse.yandex/docs/fa/interfaces/tcp/) diff --git a/docs/fa/interfaces/third-party_client_libraries.md b/docs/fa/interfaces/third-party/client_libraries.md similarity index 81% rename from docs/fa/interfaces/third-party_client_libraries.md rename to docs/fa/interfaces/third-party/client_libraries.md index 3f6536ccf22..48034195fab 100644 --- a/docs/fa/interfaces/third-party_client_libraries.md +++ b/docs/fa/interfaces/third-party/client_libraries.md @@ -1,12 +1,12 @@
-# کتابخانه های توسعه دهندگان third-party +# کتابخانه های مشتری شخص ثالث -کتابخانه هایی برای کار با ClickHouse وجود دارد: +!!! warning "سلب مسئولیت" + Yandex نه حفظ کتابخانه ها در زیر ذکر شده و نشده انجام هر آزمایش های گسترده ای برای اطمینان از کیفیت آنها. - Python - [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm) - - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) - [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-client](https://github.com/yurial/clickhouse-client) - PHP @@ -31,21 +31,23 @@ - R - [clickhouse-r](https://github.com/hannesmuehleisen/clickhouse-r) - [RClickhouse](https://github.com/IMSMWU/RClickhouse) -- .NET +- Java + - [clickhouse-client-java](https://github.com/VirtusAI/clickhouse-client-java) +- Scala + - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) +- Kotlin + - [AORM](https://github.com/TanVD/AORM) +- C# - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) - - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) - C++ - [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp/) - Elixir - [clickhousex](https://github.com/appodeal/clickhousex/) - - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) -- Java - - [clickhouse-client-java](https://github.com/VirtusAI/clickhouse-client-java) - Nim - [nim-clickhouse](https://github.com/leonardoce/nim-clickhouse) ما این کتابخانه ها را تست نکردیم. آنها به صورت تصادفی انتخاب شده اند.
-[مقاله اصلی](https://clickhouse.yandex/docs/fa/interfaces/third-party_client_libraries/) +[مقاله اصلی](https://clickhouse.yandex/docs/fa/interfaces/third-party/client_libraries/) diff --git a/docs/fa/interfaces/third-party_gui.md b/docs/fa/interfaces/third-party/gui.md similarity index 100% rename from docs/fa/interfaces/third-party_gui.md rename to docs/fa/interfaces/third-party/gui.md diff --git a/docs/fa/interfaces/third-party/integrations.md b/docs/fa/interfaces/third-party/integrations.md new file mode 100644 index 00000000000..08055497848 --- /dev/null +++ b/docs/fa/interfaces/third-party/integrations.md @@ -0,0 +1,27 @@ +
+ +# کتابخانه ادغام ثالث + +!!! warning "سلب مسئولیت" + Yandex نه حفظ کتابخانه ها در زیر ذکر شده و نشده انجام هر آزمایش های گسترده ای برای اطمینان از کیفیت آنها. + +- Python + - [SQLAlchemy](https://www.sqlalchemy.org) + - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) +- Java + - [Hadoop](http://hadoop.apache.org) + - [clickhouse-hdfs-loader](https://github.com/jaykelin/clickhouse-hdfs-loader) (uses [JDBC](../jdbc.md)) +- Scala + - [Akka](https://akka.io) + - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) +- C# + - [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview) + - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) + - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) + - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) +- Elixir + - [Ecto](https://github.com/elixir-ecto/ecto) + - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) + +
+[مقاله اصلی](https://clickhouse.yandex/docs/fa/interfaces/third-party/integrations/) diff --git a/docs/fa/introduction/features_considered_disadvantages.md b/docs/fa/introduction/features_considered_disadvantages.md index 3a963d05140..f6066bbeadf 100644 --- a/docs/fa/introduction/features_considered_disadvantages.md +++ b/docs/fa/introduction/features_considered_disadvantages.md @@ -3,7 +3,7 @@ # ویژگی های از ClickHouse که می تواند معایبی باشد. 1. بدون پشتیبانی کامل از تراکنش -2. عدم توانایی در تغییر و یا حذف داده های insert شده با rate بالا و latency کم. روشی برای پاک کردن دسته ای داده ها و یا مطابق با قوانین [GDPR](https://gdpr-info.eu) وجود دارد. بروزرسانی دسته ای از July 2018 در حال توسعه می باشد. +2. عدم توانایی برای تغییر و یا حذف داده های در حال حاضر وارد شده با سرعت بالا و تاخیر کم. برای پاک کردن و یا اصلاح داده ها، به عنوان مثال برای پیروی از [GDPR](https://gdpr-info.eu)، دسته ای پاک و به روزرسانی وجود دارد.حال توسعه می باشد. 3. Sparse index باعث می شود ClickHouse چندان مناسب اجرای پرسمان های point query برای دریافت یک ردیف از داده ها با استفاده از کلید آنها نباشد. diff --git a/docs/fa/query_language/functions/functions_for_nulls.md b/docs/fa/query_language/functions/functions_for_nulls.md new file mode 120000 index 00000000000..fa57e10ad15 --- /dev/null +++ b/docs/fa/query_language/functions/functions_for_nulls.md @@ -0,0 +1 @@ +../../../en/query_language/functions/functions_for_nulls.md \ No newline at end of file diff --git a/docs/fa/query_language/functions/geo.md b/docs/fa/query_language/functions/geo.md new file mode 120000 index 00000000000..86fa3a85d34 --- /dev/null +++ b/docs/fa/query_language/functions/geo.md @@ -0,0 +1 @@ +../../../en/query_language/functions/geo.md \ No newline at end of file diff --git a/docs/redirects.txt b/docs/redirects.txt index f9473c69d94..be807dd547d 100644 --- a/docs/redirects.txt +++ b/docs/redirects.txt @@ -116,3 +116,6 @@ table_functions/numbers.md query_language/table_functions/numbers.md table_functions/remote.md query_language/table_functions/remote.md query_language/queries.md query_language.md introduction/possible_silly_questions.md faq/general.md +interfaces/third-party_client_libraries.md interfaces/third-party/client_libraries.md +interfaces/third-party_gui.md interfaces/third-party/gui.md +interfaces/http_interface.md interfaces/http.md diff --git a/docs/ru/interfaces/http_interface.md b/docs/ru/interfaces/http.md similarity index 100% rename from docs/ru/interfaces/http_interface.md rename to docs/ru/interfaces/http.md diff --git a/docs/ru/interfaces/index.md b/docs/ru/interfaces/index.md index 4560cda1fb7..bc3b1f2bd79 100644 --- a/docs/ru/interfaces/index.md +++ b/docs/ru/interfaces/index.md @@ -2,6 +2,19 @@ # Интерфейсы -Для изучения возможностей системы, загрузки данных в таблицы, ручных запросов, используйте программу clickhouse-client. +ClickHouse предоставляет два сетевых интерфейса (оба могут быть дополнительно обернуты в TLS для дополнительной безопасности): + +* [HTTP](http.md), который задокументирован и прост для использования напрямую; +* [Native TCP](tcp.md), который имеет меньше накладных расходов. + +В большинстве случаев рекомендуется использовать подходящий инструмент или библиотеку, а не напрямую взаимодействовать с ClickHouse по сути. Официально поддерживаемые Яндексом: +* [Консольный клиент](cli.md); +* [JDBC-драйвер](jdbc.md); +* [ODBC-драйвер](odbc.md). + +Существует также широкий спектр сторонних библиотек для работы с ClickHouse: +* [Клиентские библиотеки](third-party/client_libraries.md); +* [Библиотеки для интеграции](third-party/integrations.md); +* [Визуальные интерфейсы](third-party/gui.md). [Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/) diff --git a/docs/ru/interfaces/odbc.md b/docs/ru/interfaces/odbc.md new file mode 100644 index 00000000000..9feb43f13ea --- /dev/null +++ b/docs/ru/interfaces/odbc.md @@ -0,0 +1,5 @@ +# ODBC-драйвер + +- [Официальный драйвер](https://github.com/yandex/clickhouse-jdbc). + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/odbc/) diff --git a/docs/ru/interfaces/tcp.md b/docs/ru/interfaces/tcp.md index 98672b505e4..da0ea735644 100644 --- a/docs/ru/interfaces/tcp.md +++ b/docs/ru/interfaces/tcp.md @@ -1,5 +1,5 @@ # Родной интерфейс (TCP) -Родной интерфейс используется в клиенте командной строки clickhouse-client, при межсерверном взаимодействии для распределённой обработки запроса, а также в программах на C++. Будет рассмотрен только клиент командной строки. +Нативный протокол используется в [клиенте командной строки](cli.md), для взаимодействия между серверами во время обработки распределенных запросов, а также в других программах на C++. К сожалению, у родного протокола ClickHouse пока нет формальной спецификации, но в нем можно разобраться с использованием исходного кода ClickHouse (начиная с [примерно этого места](https://github.com/yandex/ClickHouse/tree/master/dbms/src/Client)) и/или путем перехвата и анализа TCP трафика. [Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/tcp/) diff --git a/docs/ru/interfaces/third-party_client_libraries.md b/docs/ru/interfaces/third-party/client_libraries.md similarity index 81% rename from docs/ru/interfaces/third-party_client_libraries.md rename to docs/ru/interfaces/third-party/client_libraries.md index 6cebd98271d..562b863922e 100644 --- a/docs/ru/interfaces/third-party_client_libraries.md +++ b/docs/ru/interfaces/third-party/client_libraries.md @@ -1,10 +1,10 @@ -# Библиотеки от сторонних разработчиков +# Клиентские библиотеки от сторонних разработчиков -Мы не тестировали перечисленные ниже библиотеки. +!!! warning "Disclaimer" + Яндекс не поддерживает перечисленные ниже библиотеки и не проводит тщательного тестирования для проверки их качества. - Python: - [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm) - - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) - [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-client](https://github.com/yurial/clickhouse-client) - [aiochclient](https://github.com/maximdanilchenko/aiochclient) @@ -30,22 +30,20 @@ - R - [clickhouse-r](https://github.com/hannesmuehleisen/clickhouse-r) - [RClickhouse](https://github.com/IMSMWU/RClickhouse) +- Java + - [clickhouse-client-java](https://github.com/VirtusAI/clickhouse-client-java) - Scala - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) -- .NET +- Kotlin + - [AORM](https://github.com/TanVD/AORM) +- C# - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) - - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) - C++ - [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp/) - Elixir - [clickhousex](https://github.com/appodeal/clickhousex/) - - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) -- Java - - [clickhouse-client-java](https://github.com/VirtusAI/clickhouse-client-java) -- Kotlin - - [AORM](https://github.com/TanVD/AORM) - Nim - [nim-clickhouse](https://github.com/leonardoce/nim-clickhouse) -[Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/third-party_client_libraries/) +[Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/third-party/client_libraries/) diff --git a/docs/ru/interfaces/third-party_gui.md b/docs/ru/interfaces/third-party/gui.md similarity index 100% rename from docs/ru/interfaces/third-party_gui.md rename to docs/ru/interfaces/third-party/gui.md diff --git a/docs/ru/interfaces/third-party/integrations.md b/docs/ru/interfaces/third-party/integrations.md new file mode 100644 index 00000000000..e86a9fb20a8 --- /dev/null +++ b/docs/ru/interfaces/third-party/integrations.md @@ -0,0 +1,24 @@ +# Библиотеки для интеграции от сторонних разработчиков + +!!! warning "Disclaimer" + Яндекс не поддерживает перечисленные ниже библиотеки и не проводит тщательного тестирования для проверки их качества. + +- Python + - [SQLAlchemy](https://www.sqlalchemy.org) + - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) +- Java + - [Hadoop](http://hadoop.apache.org) + - [clickhouse-hdfs-loader](https://github.com/jaykelin/clickhouse-hdfs-loader) (uses [JDBC](../jdbc.md)) +- Scala + - [Akka](https://akka.io) + - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) +- C# + - [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview) + - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) + - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) + - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) +- Elixir + - [Ecto](https://github.com/elixir-ecto/ecto) + - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/third-party/integrations/) diff --git a/docs/ru/introduction/features_considered_disadvantages.md b/docs/ru/introduction/features_considered_disadvantages.md index 9e04f747c10..f7425efa4b3 100644 --- a/docs/ru/introduction/features_considered_disadvantages.md +++ b/docs/ru/introduction/features_considered_disadvantages.md @@ -1,7 +1,7 @@ # Особенности ClickHouse, которые могут считаться недостатками 1. Отсутствие полноценных транзакций. -2. Возможность изменять или удалять ранее записанные данные с низкими задержками и высокой частотой запросов не предоставляется. Есть массовое удаление данных для очистки более не нужного или соответствия [GDPR](https://gdpr-info.eu). Массовое изменение данных находится в разработке (на момент июля 2018). +2. Возможность изменять или удалять ранее записанные данные с низкими задержками и высокой частотой запросов не предоставляется. Есть массовое удаление и изменение данных для очистки более не нужного или соответствия [GDPR](https://gdpr-info.eu). 3. Разреженный индекс делает ClickHouse плохо пригодным для точечных чтений одиночных строк по своим ключам. diff --git a/docs/ru/operations/table_engines/custom_partitioning_key.md b/docs/ru/operations/table_engines/custom_partitioning_key.md index 8d228f5c498..6aa3cad3d0f 100644 --- a/docs/ru/operations/table_engines/custom_partitioning_key.md +++ b/docs/ru/operations/table_engines/custom_partitioning_key.md @@ -2,8 +2,6 @@ # Произвольный ключ партиционирования -Начиная с версии 1.1.54310 доступна возможность создания таблиц семейства MergeTree с произвольным выражением партиционирования (не только по месяцу). - Ключ партиционирования может представлять собой произвольное выражение из столбцов таблицы, а также кортеж из таких выражений (аналогично первичному ключу). Ключ партиционирования может отсутствовать. При создании таблицы ключ партиционирования указывается в описании движка (ENGINE) с новым синтаксисом: ``` diff --git a/docs/ru/roadmap.md b/docs/ru/roadmap.md deleted file mode 100644 index 5418cf8a8b1..00000000000 --- a/docs/ru/roadmap.md +++ /dev/null @@ -1,14 +0,0 @@ -# Roadmap - -## Q4 2018 - -- Соответствующий SQL стандарту синтаксис JOIN: - - Несколько `JOIN`ов в одном `SELECT` - -- Улучшения в исполнении JOIN: - - Распределённый JOIN, не ограниченный оперативной памятью - -- Добавление Protobuf и Parquet к ассортименту поддерживаемых форматов ввода-вывода -- Пулы ресурсов для более точного распределения мощностей кластера между его пользователями - -[Оригинальная статья](https://clickhouse.yandex/docs/ru/roadmap/) diff --git a/docs/ru/roadmap.md b/docs/ru/roadmap.md new file mode 120000 index 00000000000..24df86352b3 --- /dev/null +++ b/docs/ru/roadmap.md @@ -0,0 +1 @@ +../en/roadmap.md \ No newline at end of file diff --git a/docs/toc_en.yml b/docs/toc_en.yml index ec4ae4e8619..6e9a252e84e 100644 --- a/docs/toc_en.yml +++ b/docs/toc_en.yml @@ -2,37 +2,40 @@ nav: - 'Introduction': - 'Overview': 'index.md' - - 'Distinctive features of ClickHouse': 'introduction/distinctive_features.md' - - 'ClickHouse features that can be considered disadvantages': 'introduction/features_considered_disadvantages.md' + - 'Distinctive Features of ClickHouse': 'introduction/distinctive_features.md' + - 'ClickHouse Features that Can Be Considered Disadvantages': 'introduction/features_considered_disadvantages.md' - 'Performance': 'introduction/performance.md' - - 'The Yandex.Metrica task': 'introduction/ya_metrika_task.md' + - 'The Yandex.Metrica Task': 'introduction/ya_metrika_task.md' -- 'Getting started': - - 'Deploying and running': 'getting_started/index.md' - - 'Example datasets': +- 'Getting Started': + - 'Deploying and Running': 'getting_started/index.md' + - 'Example Datasets': - 'OnTime': 'getting_started/example_datasets/ontime.md' - - 'New York Taxi data': 'getting_started/example_datasets/nyc_taxi.md' + - 'New York Taxi Data': 'getting_started/example_datasets/nyc_taxi.md' - 'AMPLab Big Data Benchmark': 'getting_started/example_datasets/amplab_benchmark.md' - 'WikiStat': 'getting_started/example_datasets/wikistat.md' - - 'Terabyte click logs from Criteo': 'getting_started/example_datasets/criteo.md' + - 'Terabyte Click Logs from Criteo': 'getting_started/example_datasets/criteo.md' - 'Star Schema Benchmark': 'getting_started/example_datasets/star_schema.md' - 'Interfaces': - 'Introduction': 'interfaces/index.md' - - 'Command-line client': 'interfaces/cli.md' - - 'HTTP interface': 'interfaces/http_interface.md' - - 'JDBC driver': 'interfaces/jdbc.md' - - 'Native interface (TCP)': 'interfaces/tcp.md' - - 'Libraries from third-party developers': 'interfaces/third-party_client_libraries.md' - - 'Visual interfaces from third-party developers': 'interfaces/third-party_gui.md' - - 'Input and output formats': 'interfaces/formats.md' + - 'Command-Line Client': 'interfaces/cli.md' + - 'Native Interface (TCP)': 'interfaces/tcp.md' + - 'HTTP Interface': 'interfaces/http.md' + - 'Input and Output Formats': 'interfaces/formats.md' + - 'JDBC Driver': 'interfaces/jdbc.md' + - 'ODBC Driver': 'interfaces/odbc.md' + - 'Third-Party': + - 'Client Libraries': 'interfaces/third-party/client_libraries.md' + - 'Integrations': 'interfaces/third-party/integrations.md' + - 'Visual Interfaces': 'interfaces/third-party/gui.md' -- 'Data types': +- 'Data Types': - 'Introduction': 'data_types/index.md' - 'UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64': 'data_types/int_uint.md' - 'Float32, Float64': 'data_types/float.md' - 'Decimal': 'data_types/decimal.md' - - 'Boolean values': 'data_types/boolean.md' + - 'Boolean': 'data_types/boolean.md' - 'String': 'data_types/string.md' - 'FixedString(N)': 'data_types/fixedstring.md' - 'Date': 'data_types/date.md' @@ -42,58 +45,58 @@ nav: - 'AggregateFunction(name, types_of_arguments...)': 'data_types/nested_data_structures/aggregatefunction.md' - 'Tuple(T1, T2, ...)': 'data_types/tuple.md' - 'Nullable': 'data_types/nullable.md' - - 'Nested data structures': + - 'Nested Data Structures': - 'hidden': 'data_types/nested_data_structures/index.md' - 'Nested(Name1 Type1, Name2 Type2, ...)': 'data_types/nested_data_structures/nested.md' - - 'Special data types': + - 'Special Data Types': - 'hidden': 'data_types/special_data_types/index.md' - 'Expression': 'data_types/special_data_types/expression.md' - 'Set': 'data_types/special_data_types/set.md' - 'Nothing': 'data_types/special_data_types/nothing.md' -- 'SQL reference': +- 'SQL Reference': - 'hidden': 'query_language/index.md' - 'SELECT': 'query_language/select.md' - 'INSERT INTO': 'query_language/insert_into.md' - 'CREATE': 'query_language/create.md' - 'ALTER': 'query_language/alter.md' - - 'Other kinds of queries': 'query_language/misc.md' + - 'Other Kinds of Queries': 'query_language/misc.md' - 'Functions': - 'Introduction': 'query_language/functions/index.md' - - 'Arithmetic functions': 'query_language/functions/arithmetic_functions.md' - - 'Comparison functions': 'query_language/functions/comparison_functions.md' - - 'Logical functions': 'query_language/functions/logical_functions.md' - - 'Type conversion functions': 'query_language/functions/type_conversion_functions.md' - - 'Functions for working with dates and times': 'query_language/functions/date_time_functions.md' - - 'Functions for working with strings': 'query_language/functions/string_functions.md' - - 'Functions for searching strings': 'query_language/functions/string_search_functions.md' - - 'Functions for searching and replacing in strings': 'query_language/functions/string_replace_functions.md' - - 'Conditional functions': 'query_language/functions/conditional_functions.md' - - 'Mathematical functions': 'query_language/functions/math_functions.md' - - 'Rounding functions': 'query_language/functions/rounding_functions.md' - - 'Functions for working with arrays': 'query_language/functions/array_functions.md' - - 'Functions for splitting and merging strings and arrays': 'query_language/functions/splitting_merging_functions.md' - - 'Bit functions': 'query_language/functions/bit_functions.md' - - 'Hash functions': 'query_language/functions/hash_functions.md' - - 'Functions for generating pseudo-random numbers': 'query_language/functions/random_functions.md' - - 'Encoding functions': 'query_language/functions/encoding_functions.md' - - 'Functions for working with URLs': 'query_language/functions/url_functions.md' - - 'Functions for working with IP addresses': 'query_language/functions/ip_address_functions.md' - - 'Functions for working with JSON.': 'query_language/functions/json_functions.md' - - 'Higher-order functions': 'query_language/functions/higher_order_functions.md' - - 'Other functions': 'query_language/functions/other_functions.md' - - 'Functions for working with external dictionaries': 'query_language/functions/ext_dict_functions.md' - - 'Functions for working with Yandex.Metrica dictionaries': 'query_language/functions/ym_dict_functions.md' - - 'Functions for implementing the IN operator': 'query_language/functions/in_functions.md' - - 'arrayJoin function': 'query_language/functions/array_join.md' - - 'Functions for working with geographical coordinates': 'query_language/functions/geo.md' - - 'Functions for working with Nullable arguments': 'query_language/functions/functions_for_nulls.md' - - 'Aggregate functions': + - 'Arithmetic': 'query_language/functions/arithmetic_functions.md' + - 'Comparison': 'query_language/functions/comparison_functions.md' + - 'Logical': 'query_language/functions/logical_functions.md' + - 'Type Conversion': 'query_language/functions/type_conversion_functions.md' + - 'Working with Dates and Times': 'query_language/functions/date_time_functions.md' + - 'Working with strings': 'query_language/functions/string_functions.md' + - 'For Searching Strings': 'query_language/functions/string_search_functions.md' + - 'For Replacing in Strings': 'query_language/functions/string_replace_functions.md' + - 'Conditional ': 'query_language/functions/conditional_functions.md' + - 'Mathematical': 'query_language/functions/math_functions.md' + - 'Rounding': 'query_language/functions/rounding_functions.md' + - 'Working with Arrays': 'query_language/functions/array_functions.md' + - 'Splitting and Merging Strings and Arrays': 'query_language/functions/splitting_merging_functions.md' + - 'Bit': 'query_language/functions/bit_functions.md' + - 'Hash': 'query_language/functions/hash_functions.md' + - 'Generating Pseudo-Random Numbers': 'query_language/functions/random_functions.md' + - 'Encoding': 'query_language/functions/encoding_functions.md' + - 'Working with URLs': 'query_language/functions/url_functions.md' + - 'Working with IP Addresses': 'query_language/functions/ip_address_functions.md' + - 'Working with JSON.': 'query_language/functions/json_functions.md' + - 'Higher-Order': 'query_language/functions/higher_order_functions.md' + - 'Working with External Dictionaries': 'query_language/functions/ext_dict_functions.md' + - 'Working with Yandex.Metrica Dictionaries': 'query_language/functions/ym_dict_functions.md' + - 'Implementing the IN Operator': 'query_language/functions/in_functions.md' + - 'arrayJoin': 'query_language/functions/array_join.md' + - 'Working with geographical coordinates': 'query_language/functions/geo.md' + - 'Working with Nullable arguments': 'query_language/functions/functions_for_nulls.md' + - 'Other': 'query_language/functions/other_functions.md' + - 'Aggregate Functions': - 'Introduction': 'query_language/agg_functions/index.md' - - 'Function reference': 'query_language/agg_functions/reference.md' + - 'Reference': 'query_language/agg_functions/reference.md' - 'Aggregate function combinators': 'query_language/agg_functions/combinators.md' - 'Parametric aggregate functions': 'query_language/agg_functions/parametric_functions.md' - - 'Table functions': + - 'Table Functions': - 'Introduction': 'query_language/table_functions/index.md' - 'file': 'query_language/table_functions/file.md' - 'merge': 'query_language/table_functions/merge.md' @@ -103,31 +106,31 @@ nav: - 'jdbc': 'query_language/table_functions/jdbc.md' - 'Dictionaries': - 'Introduction': 'query_language/dicts/index.md' - - 'External dictionaries': - - 'General description': 'query_language/dicts/external_dicts.md' - - 'Configuring an external dictionary': 'query_language/dicts/external_dicts_dict.md' - - 'Storing dictionaries in memory': 'query_language/dicts/external_dicts_dict_layout.md' - - 'Dictionary updates': 'query_language/dicts/external_dicts_dict_lifetime.md' - - 'Sources of external dictionaries': 'query_language/dicts/external_dicts_dict_sources.md' - - 'Dictionary key and fields': 'query_language/dicts/external_dicts_dict_structure.md' - - 'Internal dictionaries': 'query_language/dicts/internal_dicts.md' + - 'External Dictionaries': + - 'General Description': 'query_language/dicts/external_dicts.md' + - 'Configuring an External Dictionary': 'query_language/dicts/external_dicts_dict.md' + - 'Storing Dictionaries in Memory': 'query_language/dicts/external_dicts_dict_layout.md' + - 'Dictionary Updates': 'query_language/dicts/external_dicts_dict_lifetime.md' + - 'Sources of External Dictionaries': 'query_language/dicts/external_dicts_dict_sources.md' + - 'Dictionary Key and Fields': 'query_language/dicts/external_dicts_dict_structure.md' + - 'Internal Dictionaries': 'query_language/dicts/internal_dicts.md' - 'Operators': 'query_language/operators.md' - - 'General syntax': 'query_language/syntax.md' + - 'General Syntax': 'query_language/syntax.md' - 'Operations': - 'hidden': 'operations/index.md' - - 'Table engines': + - 'Table Engines': - 'Introduction': 'operations/table_engines/index.md' - - 'MergeTree family': + - 'MergeTree Family': - 'MergeTree': 'operations/table_engines/mergetree.md' - - 'Data replication': 'operations/table_engines/replication.md' - - 'Custom partitioning key': 'operations/table_engines/custom_partitioning_key.md' + - 'Data Replication': 'operations/table_engines/replication.md' + - 'Custom Partitioning Key': 'operations/table_engines/custom_partitioning_key.md' - 'ReplacingMergeTree': 'operations/table_engines/replacingmergetree.md' - 'SummingMergeTree': 'operations/table_engines/summingmergetree.md' - 'AggregatingMergeTree': 'operations/table_engines/aggregatingmergetree.md' - 'CollapsingMergeTree': 'operations/table_engines/collapsingmergetree.md' - 'GraphiteMergeTree': 'operations/table_engines/graphitemergetree.md' - - 'For small data': + - 'For Small Data': - 'TinyLog': 'operations/table_engines/tinylog.md' - 'Log': 'operations/table_engines/log.md' - 'Memory': 'operations/table_engines/memory.md' @@ -147,37 +150,37 @@ nav: - 'Integrations': - 'Kafka': 'operations/table_engines/kafka.md' - 'MySQL': 'operations/table_engines/mysql.md' - - 'Access rights': 'operations/access_rights.md' - - 'Configuration files': 'operations/configuration_files.md' + - 'Access Rights': 'operations/access_rights.md' + - 'Configuration Files': 'operations/configuration_files.md' - 'Quotas': 'operations/quotas.md' - - 'System tables': 'operations/system_tables.md' - - 'Usage recommendations': 'operations/tips.md' - - 'Server configuration parameters': + - 'System Tables': 'operations/system_tables.md' + - 'Usage Recommendations': 'operations/tips.md' + - 'Server Configuration Parameters': - 'Introduction': 'operations/server_settings/index.md' - - 'Server settings': 'operations/server_settings/settings.md' + - 'Server Settings': 'operations/server_settings/settings.md' - 'Settings': - 'Introduction': 'operations/settings/index.md' - - 'Permissions for queries': 'operations/settings/permissions_for_queries.md' - - 'Restrictions on query complexity': 'operations/settings/query_complexity.md' + - 'Permissions for Queries': 'operations/settings/permissions_for_queries.md' + - 'Restrictions on Query Complexity': 'operations/settings/query_complexity.md' - 'Settings': 'operations/settings/settings.md' - - 'Settings profiles': 'operations/settings/settings_profiles.md' + - 'Settings Profiles': 'operations/settings/settings_profiles.md' - 'Utilities': - 'Overview': 'operations/utils/index.md' - 'clickhouse-copier': 'operations/utils/clickhouse-copier.md' - 'clickhouse-local': 'operations/utils/clickhouse-local.md' - 'F.A.Q.': - - 'General questions': 'faq/general.md' + - 'General Questions': 'faq/general.md' - 'Development': - 'hidden': 'development/index.md' - - 'Overview of ClickHouse architecture': 'development/architecture.md' - - 'How to build ClickHouse on Linux': 'development/build.md' - - 'How to build ClickHouse on Mac OS X': 'development/build_osx.md' - - 'How to write C++ code': 'development/style.md' - - 'How to run ClickHouse tests': 'development/tests.md' + - 'Overview of ClickHouse Architecture': 'development/architecture.md' + - 'How to Build ClickHouse on Linux': 'development/build.md' + - 'How to Build ClickHouse on Mac OS X': 'development/build_osx.md' + - 'How to Write C++ code': 'development/style.md' + - 'How to Run ClickHouse Tests': 'development/tests.md' -- 'What''s new': +- 'What''s New': - 'Roadmap': 'roadmap.md' - 'Changelog': 'changelog.md' - - 'Security changelog': 'security_changelog.md' + - 'Security Changelog': 'security_changelog.md' diff --git a/docs/toc_fa.yml b/docs/toc_fa.yml index bf10398ce85..cc0aa901c1a 100644 --- a/docs/toc_fa.yml +++ b/docs/toc_fa.yml @@ -20,12 +20,15 @@ nav: - 'Interfaces': - 'Interface ها': 'interfaces/index.md' - ' کلاینت Command-line': 'interfaces/cli.md' - - 'HTTP interface': 'interfaces/http_interface.md' - - ' درایور JDBC': 'interfaces/jdbc.md' - 'Native interface (TCP)': 'interfaces/tcp.md' - - ' کتابخانه های توسعه دهندگان third-party': 'interfaces/third-party_client_libraries.md' - - 'interface های visual توسعه دهندگان third-party': 'interfaces/third-party_gui.md' + - 'HTTP interface': 'interfaces/http.md' - ' فرمت های Input و Output': 'interfaces/formats.md' + - ' درایور JDBC': 'interfaces/jdbc.md' + - ' درایور ODBC': 'interfaces/odbc.md' + - 'Third-party': + - 'کتابخانه های مشتری': 'interfaces/third-party/client_libraries.md' + - 'یکپارچگی': 'interfaces/third-party/integrations.md' + - 'رابط های بصری': 'interfaces/third-party/gui.md' - 'Data types': - 'Introduction': 'data_types/index.md' @@ -59,32 +62,34 @@ nav: - 'Other kinds of queries': 'query_language/misc.md' - 'Functions': - 'Introduction': 'query_language/functions/index.md' - - 'Arithmetic functions': 'query_language/functions/arithmetic_functions.md' - - 'Comparison functions': 'query_language/functions/comparison_functions.md' - - 'Logical functions': 'query_language/functions/logical_functions.md' - - 'Type conversion functions': 'query_language/functions/type_conversion_functions.md' - - 'Functions for working with dates and times': 'query_language/functions/date_time_functions.md' - - 'Functions for working with strings': 'query_language/functions/string_functions.md' - - 'Functions for searching strings': 'query_language/functions/string_search_functions.md' - - 'Functions for searching and replacing in strings': 'query_language/functions/string_replace_functions.md' - - 'Conditional functions': 'query_language/functions/conditional_functions.md' - - 'Mathematical functions': 'query_language/functions/math_functions.md' - - 'Rounding functions': 'query_language/functions/rounding_functions.md' - - 'Functions for working with arrays': 'query_language/functions/array_functions.md' - - 'Functions for splitting and merging strings and arrays': 'query_language/functions/splitting_merging_functions.md' - - 'Bit functions': 'query_language/functions/bit_functions.md' - - 'Hash functions': 'query_language/functions/hash_functions.md' - - 'Functions for generating pseudo-random numbers': 'query_language/functions/random_functions.md' - - 'Encoding functions': 'query_language/functions/encoding_functions.md' - - 'Functions for working with URLs': 'query_language/functions/url_functions.md' - - 'Functions for working with IP addresses': 'query_language/functions/ip_address_functions.md' - - 'Functions for working with JSON.': 'query_language/functions/json_functions.md' - - 'Higher-order functions': 'query_language/functions/higher_order_functions.md' - - 'Other functions': 'query_language/functions/other_functions.md' - - 'Functions for working with external dictionaries': 'query_language/functions/ext_dict_functions.md' - - 'Functions for working with Yandex.Metrica dictionaries': 'query_language/functions/ym_dict_functions.md' - - 'Functions for implementing the IN operator': 'query_language/functions/in_functions.md' - - 'arrayJoin function': 'query_language/functions/array_join.md' + - 'Arithmetic': 'query_language/functions/arithmetic_functions.md' + - 'Comparison': 'query_language/functions/comparison_functions.md' + - 'Logical': 'query_language/functions/logical_functions.md' + - 'Type Conversion': 'query_language/functions/type_conversion_functions.md' + - 'Working with Dates and Times': 'query_language/functions/date_time_functions.md' + - 'Working with strings': 'query_language/functions/string_functions.md' + - 'For Searching Strings': 'query_language/functions/string_search_functions.md' + - 'For Replacing in Strings': 'query_language/functions/string_replace_functions.md' + - 'Conditional ': 'query_language/functions/conditional_functions.md' + - 'Mathematical': 'query_language/functions/math_functions.md' + - 'Rounding': 'query_language/functions/rounding_functions.md' + - 'Working with Arrays': 'query_language/functions/array_functions.md' + - 'Splitting and Merging Strings and Arrays': 'query_language/functions/splitting_merging_functions.md' + - 'Bit': 'query_language/functions/bit_functions.md' + - 'Hash': 'query_language/functions/hash_functions.md' + - 'Generating Pseudo-Random Numbers': 'query_language/functions/random_functions.md' + - 'Encoding': 'query_language/functions/encoding_functions.md' + - 'Working with URLs': 'query_language/functions/url_functions.md' + - 'Working with IP Addresses': 'query_language/functions/ip_address_functions.md' + - 'Working with JSON.': 'query_language/functions/json_functions.md' + - 'Higher-Order': 'query_language/functions/higher_order_functions.md' + - 'Working with External Dictionaries': 'query_language/functions/ext_dict_functions.md' + - 'Working with Yandex.Metrica Dictionaries': 'query_language/functions/ym_dict_functions.md' + - 'Implementing the IN Operator': 'query_language/functions/in_functions.md' + - 'arrayJoin': 'query_language/functions/array_join.md' + - 'Working with geographical coordinates': 'query_language/functions/geo.md' + - 'Working with Nullable arguments': 'query_language/functions/functions_for_nulls.md' + - 'Other': 'query_language/functions/other_functions.md' - 'Aggregate functions': - 'Introduction': 'query_language/agg_functions/index.md' - 'Function reference': 'query_language/agg_functions/reference.md' diff --git a/docs/toc_ru.yml b/docs/toc_ru.yml index 00003aa9359..b4f4d60fd93 100644 --- a/docs/toc_ru.yml +++ b/docs/toc_ru.yml @@ -20,12 +20,15 @@ nav: - 'Интерфейсы': - 'Введение': 'interfaces/index.md' - 'Клиент командной строки': 'interfaces/cli.md' - - 'HTTP-интерфейс': 'interfaces/http_interface.md' - - 'JDBC-драйвер': 'interfaces/jdbc.md' - - 'Родной интерфейс (TCP)': 'interfaces/tcp.md' - - 'Библиотеки от сторонних разработчиков': 'interfaces/third-party_client_libraries.md' - - 'Визуальные интерфейсы от сторонних разработчиков': 'interfaces/third-party_gui.md' + - 'Нативный интерфейс (TCP)': 'interfaces/tcp.md' + - 'HTTP-интерфейс': 'interfaces/http.md' - 'Форматы входных и выходных данных': 'interfaces/formats.md' + - 'JDBC-драйвер': 'interfaces/jdbc.md' + - 'ODBC-драйвер': 'interfaces/odbc.md' + - 'От сторонних разработчиков': + - 'Клиентские библиотеки': 'interfaces/third-party/client_libraries.md' + - 'Интеграции': 'interfaces/third-party/integrations.md' + - 'Визуальные интерфейсы': 'interfaces/third-party/gui.md' - 'Типы данных': - 'Введение': 'data_types/index.md' @@ -81,14 +84,13 @@ nav: - 'Функции для работы с IP-адресами': 'query_language/functions/ip_address_functions.md' - 'Функции для работы с JSON.': 'query_language/functions/json_functions.md' - 'Функции высшего порядка': 'query_language/functions/higher_order_functions.md' - - 'Прочие функции': 'query_language/functions/other_functions.md' - 'Функции для работы с внешними словарями': 'query_language/functions/ext_dict_functions.md' - 'Функции для работы со словарями Яндекс.Метрики': 'query_language/functions/ym_dict_functions.md' - 'Функции для реализации оператора IN.': 'query_language/functions/in_functions.md' - 'Функция arrayJoin': 'query_language/functions/array_join.md' - 'Функции для работы с географическими координатами': 'query_language/functions/geo.md' - 'Функции c Nullable агрументами': 'query_language/functions/functions_for_nulls.md' - + - 'Прочие функции': 'query_language/functions/other_functions.md' - 'Агрегатные функции': - 'Введение': 'query_language/agg_functions/index.md' - 'Справочник функций': 'query_language/agg_functions/reference.md' diff --git a/docs/toc_zh.yml b/docs/toc_zh.yml index 3b5a3d2756e..dd2811223af 100644 --- a/docs/toc_zh.yml +++ b/docs/toc_zh.yml @@ -20,12 +20,15 @@ nav: - '客户端': - '介绍': 'interfaces/index.md' - '命令行客户端接口': 'interfaces/cli.md' - - 'HTTP 客户端接口': 'interfaces/http_interface.md' - - 'JDBC 驱动': 'interfaces/jdbc.md' - '原生客户端接口 (TCP)': 'interfaces/tcp.md' - - '第三方开发的库': 'interfaces/third-party_client_libraries.md' - - '第三方开发的可视化界面': 'interfaces/third-party_gui.md' + - 'HTTP 客户端接口': 'interfaces/http.md' - '输入输出格式': 'interfaces/formats.md' + - 'JDBC 驱动': 'interfaces/jdbc.md' + - 'ODBC 驱动': 'interfaces/odbc.md' + - '第三方': + - '客户端库': 'interfaces/third-party/client_libraries.md' + - '集成': 'interfaces/third-party/integrations.md' + - '可视界面': 'interfaces/third-party/gui.md' - '数据类型': - '介绍': 'data_types/index.md' @@ -60,34 +63,34 @@ nav: - 'Other kinds of queries': 'query_language/misc.md' - 'Functions': - 'Introduction': 'query_language/functions/index.md' - - 'Arithmetic functions': 'query_language/functions/arithmetic_functions.md' - - 'Comparison functions': 'query_language/functions/comparison_functions.md' - - 'Logical functions': 'query_language/functions/logical_functions.md' - - 'Type conversion functions': 'query_language/functions/type_conversion_functions.md' - - 'Functions for working with dates and times': 'query_language/functions/date_time_functions.md' - - 'Functions for working with strings': 'query_language/functions/string_functions.md' - - 'Functions for searching strings': 'query_language/functions/string_search_functions.md' - - 'Functions for searching and replacing in strings': 'query_language/functions/string_replace_functions.md' - - 'Conditional functions': 'query_language/functions/conditional_functions.md' - - 'Mathematical functions': 'query_language/functions/math_functions.md' - - 'Rounding functions': 'query_language/functions/rounding_functions.md' - - 'Functions for working with arrays': 'query_language/functions/array_functions.md' - - 'Functions for splitting and merging strings and arrays': 'query_language/functions/splitting_merging_functions.md' - - 'Bit functions': 'query_language/functions/bit_functions.md' - - 'Hash functions': 'query_language/functions/hash_functions.md' - - 'Functions for generating pseudo-random numbers': 'query_language/functions/random_functions.md' - - 'Encoding functions': 'query_language/functions/encoding_functions.md' - - 'Functions for working with URLs': 'query_language/functions/url_functions.md' - - 'Functions for working with IP addresses': 'query_language/functions/ip_address_functions.md' - - 'Functions for working with JSON.': 'query_language/functions/json_functions.md' - - 'Higher-order functions': 'query_language/functions/higher_order_functions.md' - - 'Other functions': 'query_language/functions/other_functions.md' - - 'Functions for working with external dictionaries': 'query_language/functions/ext_dict_functions.md' - - 'Functions for working with Yandex.Metrica dictionaries': 'query_language/functions/ym_dict_functions.md' - - 'Functions for implementing the IN operator': 'query_language/functions/in_functions.md' - - 'arrayJoin function': 'query_language/functions/array_join.md' - - 'Functions for working with geographical coordinates': 'query_language/functions/geo.md' - - 'Functions for working with Nullable arguments': 'query_language/functions/functions_for_nulls.md' + - 'Arithmetic': 'query_language/functions/arithmetic_functions.md' + - 'Comparison': 'query_language/functions/comparison_functions.md' + - 'Logical': 'query_language/functions/logical_functions.md' + - 'Type Conversion': 'query_language/functions/type_conversion_functions.md' + - 'Working with Dates and Times': 'query_language/functions/date_time_functions.md' + - 'Working with strings': 'query_language/functions/string_functions.md' + - 'For Searching Strings': 'query_language/functions/string_search_functions.md' + - 'For Replacing in Strings': 'query_language/functions/string_replace_functions.md' + - 'Conditional ': 'query_language/functions/conditional_functions.md' + - 'Mathematical': 'query_language/functions/math_functions.md' + - 'Rounding': 'query_language/functions/rounding_functions.md' + - 'Working with Arrays': 'query_language/functions/array_functions.md' + - 'Splitting and Merging Strings and Arrays': 'query_language/functions/splitting_merging_functions.md' + - 'Bit': 'query_language/functions/bit_functions.md' + - 'Hash': 'query_language/functions/hash_functions.md' + - 'Generating Pseudo-Random Numbers': 'query_language/functions/random_functions.md' + - 'Encoding': 'query_language/functions/encoding_functions.md' + - 'Working with URLs': 'query_language/functions/url_functions.md' + - 'Working with IP Addresses': 'query_language/functions/ip_address_functions.md' + - 'Working with JSON.': 'query_language/functions/json_functions.md' + - 'Higher-Order': 'query_language/functions/higher_order_functions.md' + - 'Working with External Dictionaries': 'query_language/functions/ext_dict_functions.md' + - 'Working with Yandex.Metrica Dictionaries': 'query_language/functions/ym_dict_functions.md' + - 'Implementing the IN Operator': 'query_language/functions/in_functions.md' + - 'arrayJoin': 'query_language/functions/array_join.md' + - 'Working with geographical coordinates': 'query_language/functions/geo.md' + - 'Working with Nullable arguments': 'query_language/functions/functions_for_nulls.md' + - 'Other': 'query_language/functions/other_functions.md' - 'Aggregate functions': - 'Introduction': 'query_language/agg_functions/index.md' - 'Function reference': 'query_language/agg_functions/reference.md' diff --git a/docs/tools/concatenate.py b/docs/tools/concatenate.py index 6fd581f5f4c..ffc49a7fed9 100755 --- a/docs/tools/concatenate.py +++ b/docs/tools/concatenate.py @@ -44,17 +44,15 @@ def concatenate(lang, docs_path, single_page_file): # function is passed into re.sub() to process links def link_proc(matchObj): - text, link = matchObj.group().strip('[)').split('](') - if link.startswith('http') or '.jpeg' in link or '.jpg' in link or '.png' in link or '.gif' in link: + text, link = matchObj.group().strip('[)').split('](', 1) + if link.startswith('http:') or link.startswith('https:') or '.jpeg' in link or '.jpg' in link or '.png' in link or '.gif' in link: return '[' + text + '](' + link + ')' else: sharp_pos = link.find('#') if sharp_pos > -1: return '[' + text + '](' + link[sharp_pos:] + ')' else: - raise RuntimeError( - 'ERROR: Link [' + text + '](' + link + ') in file ' + - path + ' has no anchor. Please provide it.') + return '[' + text + '](#' + link.replace('../', '').replace('/index.md', '').replace('.md', '') + ')' for l in f: # Processing links in a string diff --git a/docs/zh/interfaces/http_interface.md b/docs/zh/interfaces/http.md similarity index 100% rename from docs/zh/interfaces/http_interface.md rename to docs/zh/interfaces/http.md diff --git a/docs/zh/interfaces/index.md b/docs/zh/interfaces/index.md index b5603f81e6f..5334ee2d400 100644 --- a/docs/zh/interfaces/index.md +++ b/docs/zh/interfaces/index.md @@ -2,8 +2,19 @@ # 客户端 -为了探索 ClickHouse 的能力,如导入数据到表中,或做一些手动的查询,可以使用 clickhouse-client 命令行程序来完成 +ClickHouse提供了两个网络接口(两者都可以选择包装在TLS中以提高安全性): +* [HTTP](http.md),记录在案,易于使用. +* [本地人TCP](tcp.md),这有较少的开销. +在大多数情况下,建议使用适当的工具或库,而不是直接与这些工具或库进行交互。 Yandex的官方支持如下: +* [命令行客户端](cli.md) +* [JDBC驱动程序](jdbc.md) +* [ODBC驱动程序](odbc.md) + +还有许多第三方库可供使用ClickHouse: +* [客户端库](third-party/client_libraries.md) +* [集成](third-party/integrations.md) +* [可视界面](third-party/gui.md) [来源文章](https://clickhouse.yandex/docs/zh/interfaces/) diff --git a/docs/zh/interfaces/odbc.md b/docs/zh/interfaces/odbc.md new file mode 100644 index 00000000000..120201bcc7c --- /dev/null +++ b/docs/zh/interfaces/odbc.md @@ -0,0 +1,5 @@ +# ODBC 驱动 + +- ClickHouse官方有 JDBC 的驱动。 见[这里](https://github.com/yandex/clickhouse-jdbc)。 + +[来源文章](https://clickhouse.yandex/docs/zh/interfaces/odbc/) diff --git a/docs/zh/interfaces/tcp.md b/docs/zh/interfaces/tcp.md index 742e252a25f..c60fc2e09aa 100644 --- a/docs/zh/interfaces/tcp.md +++ b/docs/zh/interfaces/tcp.md @@ -1,6 +1,5 @@ # 原生客户端接口(TCP) -TCP 原生接口用于 `clickhouse-client` 命令行,它可以在分布式查询执行中和服务器进行交互,并且可以用在 C++ 程序中。我们讲解只覆盖命令行客户端。 - +本机协议用于[命令行客户端](cli.md),用于分布式查询处理期间的服务器间通信,以及其他C ++程序。 不幸的是,本机ClickHouse协议还没有正式的规范,但它可以从ClickHouse源代码进行逆向工程[从这里开始](https://github.com/yandex/ClickHouse/tree/master/dbms/src/Client))和/或拦截和分析TCP流量。 [来源文章](https://clickhouse.yandex/docs/zh/interfaces/tcp/) diff --git a/docs/zh/interfaces/third-party_client_libraries.md b/docs/zh/interfaces/third-party/client_libraries.md similarity index 85% rename from docs/zh/interfaces/third-party_client_libraries.md rename to docs/zh/interfaces/third-party/client_libraries.md index c3cabeaca80..f0a4eba2ba3 100644 --- a/docs/zh/interfaces/third-party_client_libraries.md +++ b/docs/zh/interfaces/third-party/client_libraries.md @@ -1,10 +1,10 @@ # 第三方开发的库 -以下都是适用于 ClickHouse 的第三方库: +!!! warning "放弃" + Yandex不维护下面列出的库,也没有进行任何广泛的测试以确保其质量。 - Python - [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm) - - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) - [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-client](https://github.com/yurial/clickhouse-client) - PHP @@ -29,21 +29,21 @@ - R - [clickhouse-r](https://github.com/hannesmuehleisen/clickhouse-r) - [RClickhouse](https://github.com/IMSMWU/RClickhouse) +- Java + - [clickhouse-client-java](https://github.com/VirtusAI/clickhouse-client-java) - Scala - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) -- .NET +- Kotlin + - [AORM](https://github.com/TanVD/AORM) +- C# - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) - - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) - C++ - [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp/) - Elixir - [clickhousex](https://github.com/appodeal/clickhousex/) - - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) -- Java - - [clickhouse-client-java](https://github.com/VirtusAI/clickhouse-client-java) - Nim - [nim-clickhouse](https://github.com/leonardoce/nim-clickhouse) -[来源文章](https://clickhouse.yandex/docs/zh/interfaces/third-party_client_libraries/) +[来源文章](https://clickhouse.yandex/docs/zh/interfaces/third-party/client_libraries/) diff --git a/docs/zh/interfaces/third-party_gui.md b/docs/zh/interfaces/third-party/gui.md similarity index 100% rename from docs/zh/interfaces/third-party_gui.md rename to docs/zh/interfaces/third-party/gui.md diff --git a/docs/zh/interfaces/third-party/integrations.md b/docs/zh/interfaces/third-party/integrations.md new file mode 100644 index 00000000000..9b33f455b09 --- /dev/null +++ b/docs/zh/interfaces/third-party/integrations.md @@ -0,0 +1,25 @@ +# 第三方集成库 + +!!! warning "放弃" + Yandex不维护下面列出的库,也没有进行任何广泛的测试以确保其质量。 + +- Python + - [SQLAlchemy](https://www.sqlalchemy.org) + - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) +- Java + - [Hadoop](http://hadoop.apache.org) + - [clickhouse-hdfs-loader](https://github.com/jaykelin/clickhouse-hdfs-loader) (uses [JDBC](../jdbc.md)) +- Scala + - [Akka](https://akka.io) + - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) +- C# + - [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview) + - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) + - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) + - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) +- Elixir + - [Ecto](https://github.com/elixir-ecto/ecto) + - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) + + +[来源文章](https://clickhouse.yandex/docs/zh/interfaces/third-party/integrations/) diff --git a/docs/zh/introduction/features_considered_disadvantages.md b/docs/zh/introduction/features_considered_disadvantages.md deleted file mode 120000 index 45d3cdf563a..00000000000 --- a/docs/zh/introduction/features_considered_disadvantages.md +++ /dev/null @@ -1 +0,0 @@ -../../en/introduction/features_considered_disadvantages.md \ No newline at end of file diff --git a/docs/zh/introduction/features_considered_disadvantages.md b/docs/zh/introduction/features_considered_disadvantages.md new file mode 100644 index 00000000000..e5cd51ebdcd --- /dev/null +++ b/docs/zh/introduction/features_considered_disadvantages.md @@ -0,0 +1,7 @@ +# ClickHouse可以考虑缺点的功能 + +1. 没有完整的交易。 +2. 缺乏以高速率和低延迟修改或删除已插入数据的能力。 有批次删除和更新可用于清理或修改数据,例如符合[GDPR](https://gdpr-info.eu)。 +3. 稀疏索引使得ClickHouse不适合通过其键检索单行的点查询。 + +[来源文章](https://clickhouse.yandex/docs/zh/introduction/features_considered_disadvantages/) diff --git a/website/index.html b/website/index.html index fffeed2e211..048b46f60e4 100644 --- a/website/index.html +++ b/website/index.html @@ -31,6 +31,7 @@ Quick Start Performance Documentation + Blog Contacts @@ -90,11 +91,6 @@
-
-
- Upcoming meetup in Amsterdam on November 15 -
-

ClickHouse. Just makes you think faster.

@@ -316,6 +312,8 @@ rel="external nofollow" target="_blank">ClickHouse for Experimentation at Spotify
  • Migrating to Yandex ClickHouse by LifeStreet (machine translation from Russian)
  • +
  • ClickHouse at ContentSquare (machine translation from French)
  • How to start ClickHouse up and win the jackpot by SMI2 (machine translation from Russian)
  • Follow official Twitter account.
  • +
  • Or email Yandex ClickHouse team directly at + turn on JavaScript to see email address. + You can also fill this form to meet us in person.
  • -

    Or email ClickHouse team at Yandex directly: - turn on JavaScript to see email address, - for example if you are interested in commercial support.

    - -

    Friendly reminder: check out the documentation in English or Russian first — maybe your question is already covered. +

    Friendly reminder: check out the documentation in English, Russian, Chinese or Farsi first — maybe your question is already covered.

    Like ClickHouse?

    @@ -452,6 +449,9 @@ clickhouse-client Twitter and LinkedIn!

    +

    Also hosting ClickHouse Meetups is very appreciated, fill this form + if you are interested and we'll get in touch. + Short reports about previous meetups are published in official ClickHouse blog.

    From 499ad5eb624eb4b8e9020953a4d1f0a69420a0ca Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Tue, 27 Nov 2018 15:42:11 +0300 Subject: [PATCH 10/47] fix test (MATERIALIZED columns in both Distributed and local tables don't work) [#CLICKHOUSE-4161] --- .../integration/test_block_structure_mismatch/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/tests/integration/test_block_structure_mismatch/test.py b/dbms/tests/integration/test_block_structure_mismatch/test.py index 44989c53da5..ad96a9aa853 100644 --- a/dbms/tests/integration/test_block_structure_mismatch/test.py +++ b/dbms/tests/integration/test_block_structure_mismatch/test.py @@ -6,8 +6,8 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], with_zookeeper=True) -node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], with_zookeeper=True) +node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml']) +node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml']) #test reproducing issue https://github.com/yandex/ClickHouse/issues/3162 @pytest.fixture(scope="module") @@ -19,7 +19,7 @@ def started_cluster(): node.query(''' CREATE TABLE local_test ( t UInt64, - date Date MATERIALIZED toDate(t/1000), + date Date DEFAULT toDate(t/1000), shard UInt64, col1 String, col2 String @@ -45,6 +45,6 @@ CREATE TABLE dist_test ( cluster.shutdown() def test(started_cluster): - node1.query("INSERT INTO dist_test (t, shard, col1, col2) VALUES (1000, 1, 'foo', 'bar'), (1000, 2, 'x', 'y')") - #time.sleep(3) + node1.query("INSERT INTO local_test (t, shard, col1, col2) VALUES (1000, 0, 'x', 'y')") + node2.query("INSERT INTO local_test (t, shard, col1, col2) VALUES (1000, 1, 'foo', 'bar')") assert node1.query("SELECT col1, col2 FROM dist_test WHERE (t < 3600000) AND (col1 = 'foo') ORDER BY t ASC") == "foo\tbar\n" From dba43be6c8c9babf0f1cb6649f14e4b9ce323c7b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Nov 2018 17:59:10 +0300 Subject: [PATCH 11/47] Fixed race condition in BackgroundProcessingPool [#CLICKHOUSE-2] --- dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp index d7a0294a1ab..b61af64b3de 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -30,14 +30,15 @@ constexpr double BackgroundProcessingPool::sleep_seconds_random_part; void BackgroundProcessingPoolTaskInfo::wake() { - if (removed) - return; - Poco::Timestamp current_time; { std::unique_lock lock(pool.tasks_mutex); + /// This will ensure that iterator is valid. Must be done under the same mutex when the iterator is invalidated. + if (removed) + return; + auto next_time_to_execute = iterator->first; auto this_task_handle = iterator->second; @@ -93,6 +94,7 @@ void BackgroundProcessingPool::removeTask(const TaskHandle & task) { std::unique_lock lock(tasks_mutex); tasks.erase(task->iterator); + /// Note that the task may be still accessible through TaskHandle (shared_ptr). } } From 1ea31e0491bc7207a368cf79e1e0b845b6d15a7a Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 27 Nov 2018 18:34:21 +0300 Subject: [PATCH 12/47] Fix icu link #3631 (#3671) --- CMakeLists.txt | 2 +- cmake/Modules/FindICU.cmake | 394 ++++++++++++++++++++++++++++++ cmake/find_icu.cmake | 16 ++ cmake/find_icu4c.cmake | 21 -- dbms/CMakeLists.txt | 3 +- dbms/src/Functions/CMakeLists.txt | 3 +- 6 files changed, 413 insertions(+), 26 deletions(-) create mode 100644 cmake/Modules/FindICU.cmake create mode 100644 cmake/find_icu.cmake delete mode 100644 cmake/find_icu4c.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index 27f31c44a5b..d215c05250a 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -236,7 +236,7 @@ include(GNUInstallDirs) include (cmake/find_ssl.cmake) include (cmake/lib_name.cmake) -include (cmake/find_icu4c.cmake) +include (cmake/find_icu.cmake) include (cmake/find_boost.cmake) include (cmake/find_zlib.cmake) include (cmake/find_zstd.cmake) diff --git a/cmake/Modules/FindICU.cmake b/cmake/Modules/FindICU.cmake new file mode 100644 index 00000000000..0e61b3dcf29 --- /dev/null +++ b/cmake/Modules/FindICU.cmake @@ -0,0 +1,394 @@ +# Distributed under the OSI-approved BSD 3-Clause License. See accompanying +# file Copyright.txt or https://cmake.org/licensing for details. + +#.rst: +# FindICU +# ------- +# +# Find the International Components for Unicode (ICU) libraries and +# programs. +# +# This module supports multiple components. +# Components can include any of: ``data``, ``i18n``, ``io``, ``le``, +# ``lx``, ``test``, ``tu`` and ``uc``. +# +# Note that on Windows ``data`` is named ``dt`` and ``i18n`` is named +# ``in``; any of the names may be used, and the appropriate +# platform-specific library name will be automatically selected. +# +# This module reports information about the ICU installation in +# several variables. General variables:: +# +# ICU_VERSION - ICU release version +# ICU_FOUND - true if the main programs and libraries were found +# ICU_LIBRARIES - component libraries to be linked +# ICU_INCLUDE_DIRS - the directories containing the ICU headers +# +# Imported targets:: +# +# ICU:: +# +# Where ```` is the name of an ICU component, for example +# ``ICU::i18n``. +# +# ICU programs are reported in:: +# +# ICU_GENCNVAL_EXECUTABLE - path to gencnval executable +# ICU_ICUINFO_EXECUTABLE - path to icuinfo executable +# ICU_GENBRK_EXECUTABLE - path to genbrk executable +# ICU_ICU-CONFIG_EXECUTABLE - path to icu-config executable +# ICU_GENRB_EXECUTABLE - path to genrb executable +# ICU_GENDICT_EXECUTABLE - path to gendict executable +# ICU_DERB_EXECUTABLE - path to derb executable +# ICU_PKGDATA_EXECUTABLE - path to pkgdata executable +# ICU_UCONV_EXECUTABLE - path to uconv executable +# ICU_GENCFU_EXECUTABLE - path to gencfu executable +# ICU_MAKECONV_EXECUTABLE - path to makeconv executable +# ICU_GENNORM2_EXECUTABLE - path to gennorm2 executable +# ICU_GENCCODE_EXECUTABLE - path to genccode executable +# ICU_GENSPREP_EXECUTABLE - path to gensprep executable +# ICU_ICUPKG_EXECUTABLE - path to icupkg executable +# ICU_GENCMN_EXECUTABLE - path to gencmn executable +# +# ICU component libraries are reported in:: +# +# ICU__FOUND - ON if component was found +# ICU__LIBRARIES - libraries for component +# +# ICU datafiles are reported in:: +# +# ICU_MAKEFILE_INC - Makefile.inc +# ICU_PKGDATA_INC - pkgdata.inc +# +# Note that ```` is the uppercased name of the component. +# +# This module reads hints about search results from:: +# +# ICU_ROOT - the root of the ICU installation +# +# The environment variable ``ICU_ROOT`` may also be used; the +# ICU_ROOT variable takes precedence. +# +# The following cache variables may also be set:: +# +# ICU_

    _EXECUTABLE - the path to executable

    +# ICU_INCLUDE_DIR - the directory containing the ICU headers +# ICU__LIBRARY - the library for component +# +# .. note:: +# +# In most cases none of the above variables will require setting, +# unless multiple ICU versions are available and a specific version +# is required. +# +# Other variables one may set to control this module are:: +# +# ICU_DEBUG - Set to ON to enable debug output from FindICU. + +# Written by Roger Leigh + +set(icu_programs + gencnval + icuinfo + genbrk + icu-config + genrb + gendict + derb + pkgdata + uconv + gencfu + makeconv + gennorm2 + genccode + gensprep + icupkg + gencmn) + +set(icu_data + Makefile.inc + pkgdata.inc) + +# The ICU checks are contained in a function due to the large number +# of temporary variables needed. +function(_ICU_FIND) + # Set up search paths, taking compiler into account. Search ICU_ROOT, + # with ICU_ROOT in the environment as a fallback if unset. + if(ICU_ROOT) + list(APPEND icu_roots "${ICU_ROOT}") + else() + if(NOT "$ENV{ICU_ROOT}" STREQUAL "") + file(TO_CMAKE_PATH "$ENV{ICU_ROOT}" NATIVE_PATH) + list(APPEND icu_roots "${NATIVE_PATH}") + set(ICU_ROOT "${NATIVE_PATH}" + CACHE PATH "Location of the ICU installation" FORCE) + endif() + endif() + + # Find include directory + list(APPEND icu_include_suffixes "include") + find_path(ICU_INCLUDE_DIR + NAMES "unicode/utypes.h" + HINTS ${icu_roots} + PATH_SUFFIXES ${icu_include_suffixes} + DOC "ICU include directory") + set(ICU_INCLUDE_DIR "${ICU_INCLUDE_DIR}" PARENT_SCOPE) + + # Get version + if(ICU_INCLUDE_DIR AND EXISTS "${ICU_INCLUDE_DIR}/unicode/uvernum.h") + file(STRINGS "${ICU_INCLUDE_DIR}/unicode/uvernum.h" icu_header_str + REGEX "^#define[\t ]+U_ICU_VERSION[\t ]+\".*\".*") + + string(REGEX REPLACE "^#define[\t ]+U_ICU_VERSION[\t ]+\"([^ \\n]*)\".*" + "\\1" icu_version_string "${icu_header_str}") + set(ICU_VERSION "${icu_version_string}") + set(ICU_VERSION "${icu_version_string}" PARENT_SCOPE) + unset(icu_header_str) + unset(icu_version_string) + endif() + + if(CMAKE_SIZEOF_VOID_P EQUAL 8) + # 64-bit binary directory + set(_bin64 "bin64") + # 64-bit library directory + set(_lib64 "lib64") + endif() + + + # Find all ICU programs + list(APPEND icu_binary_suffixes "${_bin64}" "bin") + foreach(program ${icu_programs}) + string(TOUPPER "${program}" program_upcase) + set(cache_var "ICU_${program_upcase}_EXECUTABLE") + set(program_var "ICU_${program_upcase}_EXECUTABLE") + find_program("${cache_var}" "${program}" + HINTS ${icu_roots} + PATH_SUFFIXES ${icu_binary_suffixes} + DOC "ICU ${program} executable") + mark_as_advanced(cache_var) + set("${program_var}" "${${cache_var}}" PARENT_SCOPE) + endforeach() + + # Find all ICU libraries + list(APPEND icu_library_suffixes "${_lib64}" "lib") + set(ICU_REQUIRED_LIBS_FOUND ON) + foreach(component ${ICU_FIND_COMPONENTS}) + string(TOUPPER "${component}" component_upcase) + set(component_cache "ICU_${component_upcase}_LIBRARY") + set(component_cache_release "${component_cache}_RELEASE") + set(component_cache_debug "${component_cache}_DEBUG") + set(component_found "${component_upcase}_FOUND") + set(component_libnames "icu${component}") + set(component_debug_libnames "icu${component}d") + + # Special case deliberate library naming mismatches between Unix + # and Windows builds + unset(component_libnames) + unset(component_debug_libnames) + list(APPEND component_libnames "icu${component}") + list(APPEND component_debug_libnames "icu${component}d") + if(component STREQUAL "data") + list(APPEND component_libnames "icudt") + # Note there is no debug variant at present + list(APPEND component_debug_libnames "icudtd") + endif() + if(component STREQUAL "dt") + list(APPEND component_libnames "icudata") + # Note there is no debug variant at present + list(APPEND component_debug_libnames "icudatad") + endif() + if(component STREQUAL "i18n") + list(APPEND component_libnames "icuin") + list(APPEND component_debug_libnames "icuind") + endif() + if(component STREQUAL "in") + list(APPEND component_libnames "icui18n") + list(APPEND component_debug_libnames "icui18nd") + endif() + + find_library("${component_cache_release}" ${component_libnames} + HINTS ${icu_roots} + PATH_SUFFIXES ${icu_library_suffixes} + DOC "ICU ${component} library (release)") + find_library("${component_cache_debug}" ${component_debug_libnames} + HINTS ${icu_roots} + PATH_SUFFIXES ${icu_library_suffixes} + DOC "ICU ${component} library (debug)") + include(SelectLibraryConfigurations) + select_library_configurations(ICU_${component_upcase}) + mark_as_advanced("${component_cache_release}" "${component_cache_debug}") + if(${component_cache}) + set("${component_found}" ON) + list(APPEND ICU_LIBRARY "${${component_cache}}") + endif() + mark_as_advanced("${component_found}") + set("${component_cache}" "${${component_cache}}" PARENT_SCOPE) + set("${component_found}" "${${component_found}}" PARENT_SCOPE) + if(${component_found}) + if (ICU_FIND_REQUIRED_${component}) + list(APPEND ICU_LIBS_FOUND "${component} (required)") + else() + list(APPEND ICU_LIBS_FOUND "${component} (optional)") + endif() + else() + if (ICU_FIND_REQUIRED_${component}) + set(ICU_REQUIRED_LIBS_FOUND OFF) + list(APPEND ICU_LIBS_NOTFOUND "${component} (required)") + else() + list(APPEND ICU_LIBS_NOTFOUND "${component} (optional)") + endif() + endif() + endforeach() + set(_ICU_REQUIRED_LIBS_FOUND "${ICU_REQUIRED_LIBS_FOUND}" PARENT_SCOPE) + set(ICU_LIBRARY "${ICU_LIBRARY}" PARENT_SCOPE) + + # Find all ICU data files + if(CMAKE_LIBRARY_ARCHITECTURE) + list(APPEND icu_data_suffixes + "${_lib64}/${CMAKE_LIBRARY_ARCHITECTURE}/icu/${ICU_VERSION}" + "lib/${CMAKE_LIBRARY_ARCHITECTURE}/icu/${ICU_VERSION}" + "${_lib64}/${CMAKE_LIBRARY_ARCHITECTURE}/icu" + "lib/${CMAKE_LIBRARY_ARCHITECTURE}/icu") + endif() + list(APPEND icu_data_suffixes + "${_lib64}/icu/${ICU_VERSION}" + "lib/icu/${ICU_VERSION}" + "${_lib64}/icu" + "lib/icu") + foreach(data ${icu_data}) + string(TOUPPER "${data}" data_upcase) + string(REPLACE "." "_" data_upcase "${data_upcase}") + set(cache_var "ICU_${data_upcase}") + set(data_var "ICU_${data_upcase}") + find_file("${cache_var}" "${data}" + HINTS ${icu_roots} + PATH_SUFFIXES ${icu_data_suffixes} + DOC "ICU ${data} data file") + mark_as_advanced(cache_var) + set("${data_var}" "${${cache_var}}" PARENT_SCOPE) + endforeach() + + if(NOT ICU_FIND_QUIETLY) + if(ICU_LIBS_FOUND) + message(STATUS "Found the following ICU libraries:") + foreach(found ${ICU_LIBS_FOUND}) + message(STATUS " ${found}") + endforeach() + endif() + if(ICU_LIBS_NOTFOUND) + message(STATUS "The following ICU libraries were not found:") + foreach(notfound ${ICU_LIBS_NOTFOUND}) + message(STATUS " ${notfound}") + endforeach() + endif() + endif() + + if(ICU_DEBUG) + message(STATUS "--------FindICU.cmake search debug--------") + message(STATUS "ICU binary path search order: ${icu_roots}") + message(STATUS "ICU include path search order: ${icu_roots}") + message(STATUS "ICU library path search order: ${icu_roots}") + message(STATUS "----------------") + endif() +endfunction() + +_ICU_FIND() + +include(FindPackageHandleStandardArgs) +FIND_PACKAGE_HANDLE_STANDARD_ARGS(ICU + FOUND_VAR ICU_FOUND + REQUIRED_VARS ICU_INCLUDE_DIR + ICU_LIBRARY + _ICU_REQUIRED_LIBS_FOUND + VERSION_VAR ICU_VERSION + FAIL_MESSAGE "Failed to find all ICU components") + +unset(_ICU_REQUIRED_LIBS_FOUND) + +if(ICU_FOUND) + set(ICU_INCLUDE_DIRS "${ICU_INCLUDE_DIR}") + set(ICU_LIBRARIES "${ICU_LIBRARY}") + foreach(_ICU_component ${ICU_FIND_COMPONENTS}) + string(TOUPPER "${_ICU_component}" _ICU_component_upcase) + set(_ICU_component_cache "ICU_${_ICU_component_upcase}_LIBRARY") + set(_ICU_component_cache_release "ICU_${_ICU_component_upcase}_LIBRARY_RELEASE") + set(_ICU_component_cache_debug "ICU_${_ICU_component_upcase}_LIBRARY_DEBUG") + set(_ICU_component_lib "ICU_${_ICU_component_upcase}_LIBRARIES") + set(_ICU_component_found "${_ICU_component_upcase}_FOUND") + set(_ICU_imported_target "ICU::${_ICU_component}") + if(${_ICU_component_found}) + set("${_ICU_component_lib}" "${${_ICU_component_cache}}") + if(NOT TARGET ${_ICU_imported_target}) + add_library(${_ICU_imported_target} UNKNOWN IMPORTED) + if(ICU_INCLUDE_DIR) + set_target_properties(${_ICU_imported_target} PROPERTIES + INTERFACE_INCLUDE_DIRECTORIES "${ICU_INCLUDE_DIR}") + endif() + if(EXISTS "${${_ICU_component_cache}}") + set_target_properties(${_ICU_imported_target} PROPERTIES + IMPORTED_LINK_INTERFACE_LANGUAGES "CXX" + IMPORTED_LOCATION "${${_ICU_component_cache}}") + endif() + if(EXISTS "${${_ICU_component_cache_release}}") + set_property(TARGET ${_ICU_imported_target} APPEND PROPERTY + IMPORTED_CONFIGURATIONS RELEASE) + set_target_properties(${_ICU_imported_target} PROPERTIES + IMPORTED_LINK_INTERFACE_LANGUAGES_RELEASE "CXX" + IMPORTED_LOCATION_RELEASE "${${_ICU_component_cache_release}}") + endif() + if(EXISTS "${${_ICU_component_cache_debug}}") + set_property(TARGET ${_ICU_imported_target} APPEND PROPERTY + IMPORTED_CONFIGURATIONS DEBUG) + set_target_properties(${_ICU_imported_target} PROPERTIES + IMPORTED_LINK_INTERFACE_LANGUAGES_DEBUG "CXX" + IMPORTED_LOCATION_DEBUG "${${_ICU_component_cache_debug}}") + endif() + endif() + endif() + unset(_ICU_component_upcase) + unset(_ICU_component_cache) + unset(_ICU_component_lib) + unset(_ICU_component_found) + unset(_ICU_imported_target) + endforeach() +endif() + +if(ICU_DEBUG) + message(STATUS "--------FindICU.cmake results debug--------") + message(STATUS "ICU found: ${ICU_FOUND}") + message(STATUS "ICU_VERSION number: ${ICU_VERSION}") + message(STATUS "ICU_ROOT directory: ${ICU_ROOT}") + message(STATUS "ICU_INCLUDE_DIR directory: ${ICU_INCLUDE_DIR}") + message(STATUS "ICU_LIBRARIES: ${ICU_LIBRARIES}") + + foreach(program IN LISTS icu_programs) + string(TOUPPER "${program}" program_upcase) + set(program_lib "ICU_${program_upcase}_EXECUTABLE") + message(STATUS "${program} program: ${${program_lib}}") + unset(program_upcase) + unset(program_lib) + endforeach() + + foreach(data IN LISTS icu_data) + string(TOUPPER "${data}" data_upcase) + string(REPLACE "." "_" data_upcase "${data_upcase}") + set(data_lib "ICU_${data_upcase}") + message(STATUS "${data} data: ${${data_lib}}") + unset(data_upcase) + unset(data_lib) + endforeach() + + foreach(component IN LISTS ICU_FIND_COMPONENTS) + string(TOUPPER "${component}" component_upcase) + set(component_lib "ICU_${component_upcase}_LIBRARIES") + set(component_found "${component_upcase}_FOUND") + message(STATUS "${component} library found: ${${component_found}}") + message(STATUS "${component} library: ${${component_lib}}") + unset(component_upcase) + unset(component_lib) + unset(component_found) + endforeach() + message(STATUS "----------------") +endif() + +unset(icu_programs) diff --git a/cmake/find_icu.cmake b/cmake/find_icu.cmake new file mode 100644 index 00000000000..e677b11b037 --- /dev/null +++ b/cmake/find_icu.cmake @@ -0,0 +1,16 @@ +option (ENABLE_ICU "Enable ICU" ON) + +if (ENABLE_ICU) + find_package(ICU COMPONENTS data i18n uc) # TODO: remove Modules/FindICU.cmake after cmake 3.7 + #set (ICU_LIBRARIES ${ICU_I18N_LIBRARY} ${ICU_UC_LIBRARY} ${ICU_DATA_LIBRARY} CACHE STRING "") + set (ICU_LIBRARIES ICU::i18n ICU::uc ICU::data CACHE STRING "") + if (ICU_FOUND) + set(USE_ICU 1) + endif () +endif () + +if (USE_ICU) + message (STATUS "Using icu=${USE_ICU}: ${ICU_INCLUDE_DIR} : ${ICU_LIBRARIES}") +else () + message (STATUS "Build without ICU (support for collations and charset conversion functions will be disabled)") +endif () diff --git a/cmake/find_icu4c.cmake b/cmake/find_icu4c.cmake deleted file mode 100644 index e39f1c4c1dd..00000000000 --- a/cmake/find_icu4c.cmake +++ /dev/null @@ -1,21 +0,0 @@ -option (ENABLE_ICU "Enable ICU" ON) - -if (ENABLE_ICU) - set (ICU_PATHS "/usr/local/opt/icu4c/lib") - set (ICU_INCLUDE_PATHS "/usr/local/opt/icu4c/include") - find_library (ICUI18N icui18n PATHS ${ICU_PATHS}) - find_library (ICUUC icuuc PATHS ${ICU_PATHS}) - find_library (ICUDATA icudata PATHS ${ICU_PATHS}) - set (ICU_LIBS ${ICUI18N} ${ICUUC} ${ICUDATA}) - - find_path (ICU_INCLUDE_DIR NAMES unicode/unistr.h PATHS ${ICU_INCLUDE_PATHS}) - if (ICU_INCLUDE_DIR AND ICU_LIBS) - set(USE_ICU 1) - endif () -endif () - -if (USE_ICU) - message (STATUS "Using icu=${USE_ICU}: ${ICU_INCLUDE_DIR} : ${ICU_LIBS}") -else () - message (STATUS "Build without ICU (support for collations and charset conversion functions will be disabled)") -endif () diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index eebd3504192..f0116d768e5 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -242,8 +242,7 @@ endif() target_link_libraries (dbms PRIVATE ${Poco_Foundation_LIBRARY}) if (USE_ICU) - target_link_libraries (dbms PRIVATE ${ICU_LIBS}) - target_include_directories (dbms SYSTEM PRIVATE ${ICU_INCLUDE_DIR}) + target_link_libraries (dbms PRIVATE ${ICU_LIBRARIES}) endif () if (USE_CAPNP) diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index 1a1d6f06f64..b9567bed775 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -24,8 +24,7 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELW endif () if (USE_ICU) - #target_link_libraries (clickhouse_functions ${ICU_LIBS}) - target_include_directories (clickhouse_functions SYSTEM PRIVATE ${ICU_INCLUDE_DIR}) + target_link_libraries (clickhouse_functions PRIVATE ${ICU_LIBRARIES}) endif () if (USE_VECTORCLASS) From f1791e94e209b238b28a89c25e3a5f81882785a6 Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 27 Nov 2018 19:11:46 +0300 Subject: [PATCH 13/47] Do not write preprocessed configs to /etc/ (#2443) --- CHANGELOG.draft.md | 3 + .../extract-from-config/ExtractFromConfig.cpp | 8 +-- dbms/programs/local/LocalServer.cpp | 8 ++- dbms/programs/server/Server.cpp | 8 ++- dbms/src/Client/Connection.cpp | 1 - dbms/src/Common/Config/ConfigProcessor.cpp | 59 +++++++++++++++---- dbms/src/Common/Config/ConfigProcessor.h | 15 ++++- dbms/src/Common/Config/ConfigReloader.cpp | 4 +- dbms/src/Common/Config/ConfigReloader.h | 2 + .../tests/zk_many_watches_reconnect.cpp | 2 +- dbms/src/Core/Defines.h | 2 + dbms/src/Interpreters/ExternalLoader.cpp | 11 ++-- dbms/src/Interpreters/ExternalLoader.h | 4 +- .../ExternalLoaderConfigRepository.cpp | 4 +- .../ExternalLoaderConfigRepository.h | 2 +- .../IExternalLoaderConfigRepository.h | 2 +- dbms/src/Interpreters/tests/users.cpp | 4 +- debian/clickhouse-client.postinst | 3 - debian/clickhouse-server.cron.d | 2 +- debian/clickhouse-server.init | 6 +- debian/clickhouse-server.postinst | 11 ++-- debian/clickhouse-server.preinst | 2 +- debian/clickhouse-server.prerm | 2 +- debian/clickhouse-server.service | 2 - debian/pbuilder-hooks/A00ccache | 6 +- debian/pbuilder-hooks/B90test-server | 20 +++---- debian/pbuilder-hooks/C99kill-make | 2 +- libs/libdaemon/include/daemon/BaseDaemon.h | 2 +- libs/libdaemon/src/BaseDaemon.cpp | 7 ++- utils/config-processor/config-processor.cpp | 4 +- 30 files changed, 133 insertions(+), 75 deletions(-) diff --git a/CHANGELOG.draft.md b/CHANGELOG.draft.md index 50a7665dc63..0674a4193fe 100644 --- a/CHANGELOG.draft.md +++ b/CHANGELOG.draft.md @@ -1 +1,4 @@ * Настройка `enable_optimize_predicate_expression` выключена по-умолчанию. + +### Улучшения: +* Файлы *-preprocessed.xml записываются в директорию с данными (/var/lib/clickhouse/preprocessed_configs). Для /etc/clickhouse-server больше не нужен +w для пользователя clickhouse. Для удобства создан симлинк /var/lib/clickhouse/preprocessed_configs -> /etc/clickhouse-server/preprocessed diff --git a/dbms/programs/extract-from-config/ExtractFromConfig.cpp b/dbms/programs/extract-from-config/ExtractFromConfig.cpp index 7f0122256dd..af9550e4547 100644 --- a/dbms/programs/extract-from-config/ExtractFromConfig.cpp +++ b/dbms/programs/extract-from-config/ExtractFromConfig.cpp @@ -26,18 +26,18 @@ static void setupLogging(const std::string & log_level) static std::string extractFromConfig( const std::string & config_path, const std::string & key, bool process_zk_includes, bool try_get = false) { - ConfigProcessor processor(config_path, /* throw_on_bad_incl = */ false, /* log_to_console = */ false); + DB::ConfigProcessor processor(config_path, /* throw_on_bad_incl = */ false, /* log_to_console = */ false); bool has_zk_includes; - XMLDocumentPtr config_xml = processor.processConfig(&has_zk_includes); + DB::XMLDocumentPtr config_xml = processor.processConfig(&has_zk_includes); if (has_zk_includes && process_zk_includes) { - ConfigurationPtr bootstrap_configuration(new Poco::Util::XMLConfiguration(config_xml)); + DB::ConfigurationPtr bootstrap_configuration(new Poco::Util::XMLConfiguration(config_xml)); zkutil::ZooKeeperPtr zookeeper = std::make_shared( *bootstrap_configuration, "zookeeper"); zkutil::ZooKeeperNodeCache zk_node_cache([&] { return zookeeper; }); config_xml = processor.processConfig(&has_zk_includes, &zk_node_cache); } - ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(config_xml)); + DB::ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(config_xml)); // do not throw exception if not found if (try_get) return configuration->getString(key, ""); diff --git a/dbms/programs/local/LocalServer.cpp b/dbms/programs/local/LocalServer.cpp index 606ce6b5b37..0dab224c7f1 100644 --- a/dbms/programs/local/LocalServer.cpp +++ b/dbms/programs/local/LocalServer.cpp @@ -115,9 +115,11 @@ try /// Load config files if exists if (config().has("config-file") || Poco::File("config.xml").exists()) { - ConfigProcessor config_processor(config().getString("config-file", "config.xml"), false, true); + const auto config_path = config().getString("config-file", "config.xml"); + ConfigProcessor config_processor(config_path, false, true); + config_processor.setConfigPath(Poco::Path(config_path).makeParent().toString()); auto loaded_config = config_processor.loadConfig(); - config_processor.savePreprocessedConfig(loaded_config); + config_processor.savePreprocessedConfig(loaded_config, loaded_config.configuration->getString("path", DBMS_DEFAULT_PATH)); config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false); } @@ -348,7 +350,7 @@ void LocalServer::setupUsers() const auto users_config_path = config().getString("users_config", config().getString("config-file", "config.xml")); ConfigProcessor config_processor(users_config_path); const auto loaded_config = config_processor.loadConfig(); - config_processor.savePreprocessedConfig(loaded_config); + config_processor.savePreprocessedConfig(loaded_config, config().getString("path", DBMS_DEFAULT_PATH)); users_config = loaded_config.configuration; } else diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 03d9a8268c0..8d3bff9ebfb 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -96,7 +96,7 @@ void Server::initialize(Poco::Util::Application & self) std::string Server::getDefaultCorePath() const { - return getCanonicalPath(config().getString("path")) + "cores"; + return getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH)) + "cores"; } int Server::main(const std::vector & /*args*/) @@ -129,7 +129,7 @@ int Server::main(const std::vector & /*args*/) ConfigProcessor config_processor(config_path); loaded_config = config_processor.loadConfigWithZooKeeperIncludes( main_config_zk_node_cache, /* fallback_to_preprocessed = */ true); - config_processor.savePreprocessedConfig(loaded_config); + config_processor.savePreprocessedConfig(loaded_config, config().getString("path", DBMS_DEFAULT_PATH)); config().removeConfiguration(old_configuration.get()); config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false); } @@ -160,7 +160,7 @@ int Server::main(const std::vector & /*args*/) } #endif - std::string path = getCanonicalPath(config().getString("path")); + std::string path = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH)); std::string default_database = config().getString("default_database", "default"); global_context->setPath(path); @@ -301,6 +301,7 @@ int Server::main(const std::vector & /*args*/) std::string include_from_path = config().getString("include_from", "/etc/metrika.xml"); auto main_config_reloader = std::make_unique(config_path, include_from_path, + config().getString("path", ""), std::move(main_config_zk_node_cache), [&](ConfigurationPtr config) { @@ -322,6 +323,7 @@ int Server::main(const std::vector & /*args*/) } auto users_config_reloader = std::make_unique(users_config_path, include_from_path, + config().getString("path", ""), zkutil::ZooKeeperNodeCache([&] { return global_context->getZooKeeper(); }), [&](ConfigurationPtr config) { global_context->setUsersConfig(config); }, /* already_loaded = */ false); diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 60fd070064b..2c704e0aa54 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -27,7 +27,6 @@ #include #endif - namespace CurrentMetrics { extern const Metric SendExternalTables; diff --git a/dbms/src/Common/Config/ConfigProcessor.cpp b/dbms/src/Common/Config/ConfigProcessor.cpp index cc7660f9641..fa8f3867e73 100644 --- a/dbms/src/Common/Config/ConfigProcessor.cpp +++ b/dbms/src/Common/Config/ConfigProcessor.cpp @@ -20,6 +20,11 @@ using namespace Poco::XML; +namespace DB +{ + +/// For cutting prerpocessed path to this base +std::string main_config_path; /// Extracts from a string the first encountered number consisting of at least two digits. static std::string numberFromHost(const std::string & s) @@ -40,13 +45,6 @@ static std::string numberFromHost(const std::string & s) return ""; } -static std::string preprocessedConfigPath(const std::string & path) -{ - Poco::Path preprocessed_path(path); - preprocessed_path.setBaseName(preprocessed_path.getBaseName() + PREPROCESSED_SUFFIX); - return preprocessed_path.toString(); -} - bool ConfigProcessor::isPreprocessedFile(const std::string & path) { return endsWith(Poco::Path(path).getBaseName(), PREPROCESSED_SUFFIX); @@ -59,7 +57,6 @@ ConfigProcessor::ConfigProcessor( bool log_to_console, const Substitutions & substitutions_) : path(path_) - , preprocessed_path(preprocessedConfigPath(path)) , throw_on_bad_incl(throw_on_bad_incl_) , substitutions(substitutions_) /// We need larger name pool to allow to support vast amount of users in users.xml files for ClickHouse. @@ -522,7 +519,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfig(bool allow_zk_includes ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(config_xml)); - return LoadedConfig{configuration, has_zk_includes, /* loaded_from_preprocessed = */ false, config_xml}; + return LoadedConfig{configuration, has_zk_includes, /* loaded_from_preprocessed = */ false, config_xml, path}; } ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( @@ -556,11 +553,44 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(config_xml)); - return LoadedConfig{configuration, has_zk_includes, !processed_successfully, config_xml}; + return LoadedConfig{configuration, has_zk_includes, !processed_successfully, config_xml, path}; } -void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config) +void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, std::string preprocessed_dir) { + if (preprocessed_path.empty()) + { + auto new_path = loaded_config.config_path; + if (new_path.substr(0, main_config_path.size()) == main_config_path) + new_path.replace(0, main_config_path.size(), ""); + std::replace(new_path.begin(), new_path.end(), '/', '_'); + + if (preprocessed_dir.empty()) + { + if (!loaded_config.configuration->has("path")) + { + // Will use current directory + auto parent_path = Poco::Path(loaded_config.config_path).makeParent(); + preprocessed_dir = parent_path.toString(); + Poco::Path poco_new_path(new_path); + poco_new_path.setBaseName(poco_new_path.getBaseName() + PREPROCESSED_SUFFIX); + new_path = poco_new_path.toString(); + } + else + { + preprocessed_dir = loaded_config.configuration->getString("path") + "/preprocessed_configs/"; + } + } + else + { + preprocessed_dir += "/preprocessed_configs/"; + } + + preprocessed_path = preprocessed_dir + new_path; + auto path = Poco::Path(preprocessed_path).makeParent(); + if (!path.toString().empty()) + Poco::File(path).createDirectories(); + } try { DOMWriter().writeNode(preprocessed_path, loaded_config.preprocessed_xml); @@ -570,3 +600,10 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config) LOG_WARNING(log, "Couldn't save preprocessed config to " << preprocessed_path << ": " << e.displayText()); } } + +void ConfigProcessor::setConfigPath(const std::string & config_path) +{ + main_config_path = config_path; +} + +} diff --git a/dbms/src/Common/Config/ConfigProcessor.h b/dbms/src/Common/Config/ConfigProcessor.h index 8663ecb682f..af227c554fa 100644 --- a/dbms/src/Common/Config/ConfigProcessor.h +++ b/dbms/src/Common/Config/ConfigProcessor.h @@ -24,6 +24,9 @@ namespace zkutil class ZooKeeperNodeCache; } +namespace DB +{ + using ConfigurationPtr = Poco::AutoPtr; using XMLDocumentPtr = Poco::AutoPtr; @@ -72,6 +75,7 @@ public: bool has_zk_includes; bool loaded_from_preprocessed; XMLDocumentPtr preprocessed_xml; + std::string config_path; }; /// If allow_zk_includes is true, expect that the configuration XML can contain from_zk nodes. @@ -85,7 +89,12 @@ public: zkutil::ZooKeeperNodeCache & zk_node_cache, bool fallback_to_preprocessed = false); - void savePreprocessedConfig(const LoadedConfig & loaded_config); + /// Save preprocessed config to specified directory. + /// If preprocessed_dir is empty - calculate from loaded_config.path + /preprocessed_configs/ + void savePreprocessedConfig(const LoadedConfig & loaded_config, std::string preprocessed_dir); + + /// Set path of main config.xml . It will be cutted from all configs placed to preprocessed_configs/ + void setConfigPath(const std::string & config_path); public: using Files = std::vector; @@ -99,7 +108,7 @@ public: private: const std::string path; - const std::string preprocessed_path; + std::string preprocessed_path; bool throw_on_bad_incl; @@ -127,3 +136,5 @@ private: zkutil::ZooKeeperNodeCache * zk_node_cache, std::unordered_set & contributing_zk_paths); }; + +} diff --git a/dbms/src/Common/Config/ConfigReloader.cpp b/dbms/src/Common/Config/ConfigReloader.cpp index f798569d0c0..41ea22b65de 100644 --- a/dbms/src/Common/Config/ConfigReloader.cpp +++ b/dbms/src/Common/Config/ConfigReloader.cpp @@ -15,10 +15,12 @@ constexpr decltype(ConfigReloader::reload_interval) ConfigReloader::reload_inter ConfigReloader::ConfigReloader( const std::string & path_, const std::string & include_from_path_, + const std::string & preprocessed_dir_, zkutil::ZooKeeperNodeCache && zk_node_cache_, Updater && updater_, bool already_loaded) : path(path_), include_from_path(include_from_path_) + , preprocessed_dir(preprocessed_dir_) , zk_node_cache(std::move(zk_node_cache_)) , updater(std::move(updater_)) { @@ -98,7 +100,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac tryLogCurrentException(log, "Error loading config from `" + path + "'"); return; } - config_processor.savePreprocessedConfig(loaded_config); + config_processor.savePreprocessedConfig(loaded_config, preprocessed_dir); /** We should remember last modification time if and only if config was sucessfully loaded * Otherwise a race condition could occur during config files update: diff --git a/dbms/src/Common/Config/ConfigReloader.h b/dbms/src/Common/Config/ConfigReloader.h index 63ed18a9105..cb5ecdae7da 100644 --- a/dbms/src/Common/Config/ConfigReloader.h +++ b/dbms/src/Common/Config/ConfigReloader.h @@ -33,6 +33,7 @@ public: ConfigReloader( const std::string & path, const std::string & include_from_path, + const std::string & preprocessed_dir, zkutil::ZooKeeperNodeCache && zk_node_cache, Updater && updater, bool already_loaded); @@ -70,6 +71,7 @@ private: std::string path; std::string include_from_path; + std::string preprocessed_dir; FilesChangesTracker files; zkutil::ZooKeeperNodeCache zk_node_cache; diff --git a/dbms/src/Common/ZooKeeper/tests/zk_many_watches_reconnect.cpp b/dbms/src/Common/ZooKeeper/tests/zk_many_watches_reconnect.cpp index b1d1d5d3101..c707d91d302 100644 --- a/dbms/src/Common/ZooKeeper/tests/zk_many_watches_reconnect.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zk_many_watches_reconnect.cpp @@ -23,7 +23,7 @@ int main(int argc, char ** argv) return 3; } - ConfigProcessor processor(argv[1], false, true); + DB::ConfigProcessor processor(argv[1], false, true); auto config = processor.loadConfig().configuration; zkutil::ZooKeeper zk(*config, "zookeeper"); zkutil::EventPtr watch = std::make_shared(); diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index 570c108b2a6..0fd332113e7 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -66,6 +66,8 @@ /// the number is unmotivated #define DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT 15 +#define DBMS_DEFAULT_PATH "/var/lib/clickhouse/" + // more aliases: https://mailman.videolan.org/pipermail/x264-devel/2014-May/010660.html #if defined(_MSC_VER) diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 2827e9ea283..8e0fe1436d9 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -1,4 +1,5 @@ -#include +#include "ExternalLoader.h" +#include #include #include #include @@ -42,12 +43,12 @@ void ExternalLoader::reloadPeriodically() } -ExternalLoader::ExternalLoader(const Poco::Util::AbstractConfiguration & config, +ExternalLoader::ExternalLoader(const Poco::Util::AbstractConfiguration & config_main, const ExternalLoaderUpdateSettings & update_settings, const ExternalLoaderConfigSettings & config_settings, std::unique_ptr config_repository, Logger * log, const std::string & loadable_object_name) - : config(config) + : config_main(config_main) , update_settings(update_settings) , config_settings(config_settings) , config_repository(std::move(config_repository)) @@ -214,7 +215,7 @@ void ExternalLoader::reloadAndUpdate(bool throw_on_error) void ExternalLoader::reloadFromConfigFiles(const bool throw_on_error, const bool force_reload, const std::string & only_dictionary) { - const auto config_paths = config_repository->list(config, config_settings.path_setting_name); + const auto config_paths = config_repository->list(config_main, config_settings.path_setting_name); for (const auto & config_path : config_paths) { @@ -262,7 +263,7 @@ void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const const auto last_modified = config_repository->getLastModificationTime(config_path); if (force_reload || last_modified > config_last_modified) { - auto loaded_config = config_repository->load(config_path); + auto loaded_config = config_repository->load(config_path, config_main.getString("path", DBMS_DEFAULT_PATH)); loadable_objects_defined_in_config[config_path].clear(); diff --git a/dbms/src/Interpreters/ExternalLoader.h b/dbms/src/Interpreters/ExternalLoader.h index 76d6cfc6f96..7a27f8a81da 100644 --- a/dbms/src/Interpreters/ExternalLoader.h +++ b/dbms/src/Interpreters/ExternalLoader.h @@ -91,7 +91,7 @@ public: using ObjectsMap = std::unordered_map; /// Objects will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds. - ExternalLoader(const Configuration & config, + ExternalLoader(const Configuration & config_main, const ExternalLoaderUpdateSettings & update_settings, const ExternalLoaderConfigSettings & config_settings, std::unique_ptr config_repository, @@ -151,7 +151,7 @@ private: pcg64 rnd_engine{randomSeed()}; - const Configuration & config; + const Configuration & config_main; const ExternalLoaderUpdateSettings & update_settings; const ExternalLoaderConfigSettings & config_settings; diff --git a/dbms/src/Interpreters/ExternalLoaderConfigRepository.cpp b/dbms/src/Interpreters/ExternalLoaderConfigRepository.cpp index bfc7965a5ad..bb8dd61ee2d 100644 --- a/dbms/src/Interpreters/ExternalLoaderConfigRepository.cpp +++ b/dbms/src/Interpreters/ExternalLoaderConfigRepository.cpp @@ -61,11 +61,11 @@ Poco::Timestamp ExternalLoaderConfigRepository::getLastModificationTime( } Poco::AutoPtr ExternalLoaderConfigRepository::load( - const std::string & config_file) const + const std::string & config_file, const std::string & preprocessed_dir) const { ConfigProcessor config_processor{config_file}; ConfigProcessor::LoadedConfig preprocessed = config_processor.loadConfig(); - config_processor.savePreprocessedConfig(preprocessed); + config_processor.savePreprocessedConfig(preprocessed, preprocessed_dir); return preprocessed.configuration; } diff --git a/dbms/src/Interpreters/ExternalLoaderConfigRepository.h b/dbms/src/Interpreters/ExternalLoaderConfigRepository.h index dd5c53329b5..a1b1606dd2c 100644 --- a/dbms/src/Interpreters/ExternalLoaderConfigRepository.h +++ b/dbms/src/Interpreters/ExternalLoaderConfigRepository.h @@ -19,7 +19,7 @@ public: Poco::Timestamp getLastModificationTime(const std::string & config_file) const override; - Poco::AutoPtr load(const std::string & config_file) const override; + Poco::AutoPtr load(const std::string & config_file, const std::string & preprocessed_dir = "") const override; }; } diff --git a/dbms/src/Interpreters/IExternalLoaderConfigRepository.h b/dbms/src/Interpreters/IExternalLoaderConfigRepository.h index 79615780242..d0caaf1b921 100644 --- a/dbms/src/Interpreters/IExternalLoaderConfigRepository.h +++ b/dbms/src/Interpreters/IExternalLoaderConfigRepository.h @@ -23,7 +23,7 @@ public: virtual Poco::Timestamp getLastModificationTime(const std::string & config_file) const = 0; - virtual Poco::AutoPtr load(const std::string & config_file) const = 0; + virtual Poco::AutoPtr load(const std::string & config_file, const std::string & preprocessed_dir = "") const = 0; virtual ~IExternalLoaderConfigRepository() {} }; diff --git a/dbms/src/Interpreters/tests/users.cpp b/dbms/src/Interpreters/tests/users.cpp index f4905398262..af07629ef62 100644 --- a/dbms/src/Interpreters/tests/users.cpp +++ b/dbms/src/Interpreters/tests/users.cpp @@ -189,11 +189,11 @@ void runOneTest(const TestDescriptor & test_descriptor) const auto path_name = createTmpPath("users.xml"); createFile(path_name, test_descriptor.config_content); - ConfigurationPtr config; + DB::ConfigurationPtr config; try { - config = ConfigProcessor(path_name).loadConfig().configuration; + config = DB::ConfigProcessor(path_name).loadConfig().configuration; } catch (const Poco::Exception & ex) { diff --git a/debian/clickhouse-client.postinst b/debian/clickhouse-client.postinst index ff54e3a58fc..38d2450dcbe 100644 --- a/debian/clickhouse-client.postinst +++ b/debian/clickhouse-client.postinst @@ -4,6 +4,3 @@ set -e CLICKHOUSE_USER=${CLICKHOUSE_USER=clickhouse} mkdir -p /etc/clickhouse-client/conf.d - -# user created by clickhouse-server package -chown -R ${CLICKHOUSE_USER} /etc/clickhouse-client || true diff --git a/debian/clickhouse-server.cron.d b/debian/clickhouse-server.cron.d index 90431886613..03bbd620aa7 100644 --- a/debian/clickhouse-server.cron.d +++ b/debian/clickhouse-server.cron.d @@ -1 +1 @@ -#*/10 * * * * root (which service > /dev/null 2>&1 && (service clickhouse-server condstart || true)) || /etc/init.d/clickhouse-server condstart > /dev/null 2>&1 +#*/10 * * * * root (which service > /dev/null 2>&1 && (service clickhouse-server condstart ||:)) || /etc/init.d/clickhouse-server condstart > /dev/null 2>&1 diff --git a/debian/clickhouse-server.init b/debian/clickhouse-server.init index 0df35f55952..9044567b2bd 100755 --- a/debian/clickhouse-server.init +++ b/debian/clickhouse-server.init @@ -100,10 +100,6 @@ check_config() initdb() { - if [ -d ${SYSCONFDIR} ]; then - su -s /bin/sh ${CLICKHOUSE_USER} -c "test -w ${SYSCONFDIR}" || chown ${CLICKHOUSE_USER}:${CLICKHOUSE_GROUP} ${SYSCONFDIR} - fi - if [ -x "$BINDIR/$EXTRACT_FROM_CONFIG" ]; then CLICKHOUSE_DATADIR_FROM_CONFIG=$(su -s $SHELL ${CLICKHOUSE_USER} -c "$BINDIR/$EXTRACT_FROM_CONFIG --config-file=\"$CLICKHOUSE_CONFIG\" --key=path") if [ "(" "$?" -ne "0" ")" -o "(" -z "${CLICKHOUSE_DATADIR_FROM_CONFIG}" ")" ]; then @@ -128,7 +124,7 @@ initdb() fi if ! $(su -s $SHELL ${CLICKHOUSE_USER} -c "test -O \"${CLICKHOUSE_DATADIR_FROM_CONFIG}\" && test -G \"${CLICKHOUSE_DATADIR_FROM_CONFIG}\""); then - if [ $(dirname "${CLICKHOUSE_DATADIR_FROM_CONFIG}") == "/" ]; then + if [ $(dirname "${CLICKHOUSE_DATADIR_FROM_CONFIG}") = "/" ]; then echo "Directory ${CLICKHOUSE_DATADIR_FROM_CONFIG} seems too dangerous to chown." else if [ ! -e "${CLICKHOUSE_DATADIR_FROM_CONFIG}" ]; then diff --git a/debian/clickhouse-server.postinst b/debian/clickhouse-server.postinst index 53e9248b71f..e17c47df6dd 100644 --- a/debian/clickhouse-server.postinst +++ b/debian/clickhouse-server.postinst @@ -9,7 +9,7 @@ CLICKHOUSE_LOGDIR=${CLICKHOUSE_LOGDIR=/var/log/clickhouse-server} CLICKHOUSE_BINDIR=${CLICKHOUSE_BINDIR=/usr/bin} CLICKHOUSE_GENERIC_PROGRAM=${CLICKHOUSE_GENERIC_PROGRAM=clickhouse} -OS=${OS=`lsb_release -is 2>/dev/null || uname -s || true`} +OS=${OS=`lsb_release -is 2>/dev/null || uname -s ||:`} test -f /etc/default/clickhouse && . /etc/default/clickhouse @@ -68,9 +68,6 @@ Please fix this and reinstall this package." >&2 exit 1 fi - if [ -d ${CLICKHOUSE_CONFDIR} ]; then - su -s /bin/sh ${CLICKHOUSE_USER} -c "test -w ${CLICKHOUSE_CONFDIR}" || chown ${CLICKHOUSE_USER}:${CLICKHOUSE_GROUP} ${CLICKHOUSE_CONFDIR} - fi if [ ! -d ${CLICKHOUSE_DATADIR} ]; then mkdir -p ${CLICKHOUSE_DATADIR} @@ -78,6 +75,12 @@ Please fix this and reinstall this package." >&2 chmod 700 ${CLICKHOUSE_DATADIR} fi + if [ -d ${CLICKHOUSE_CONFDIR} ]; then + rm -v ${CLICKHOUSE_CONFDIR}/*-preprocessed.xml ||: + fi + + ln -s ${CLICKHOUSE_DATADIR}/preprocessed_configs ${CLICKHOUSE_CONFDIR}/preprocessed ||: + if [ ! -d ${CLICKHOUSE_LOGDIR} ]; then mkdir -p ${CLICKHOUSE_LOGDIR} chown root:${CLICKHOUSE_GROUP} ${CLICKHOUSE_LOGDIR} diff --git a/debian/clickhouse-server.preinst b/debian/clickhouse-server.preinst index 603a67a12a5..1435d8db8da 100644 --- a/debian/clickhouse-server.preinst +++ b/debian/clickhouse-server.preinst @@ -2,5 +2,5 @@ if [ "$1" = "upgrade" ]; then # Return etc/cron.d/clickhouse-server to original state - service clickhouse-server disable_cron || true + service clickhouse-server disable_cron ||: fi diff --git a/debian/clickhouse-server.prerm b/debian/clickhouse-server.prerm index d9846c38085..02e855a7125 100644 --- a/debian/clickhouse-server.prerm +++ b/debian/clickhouse-server.prerm @@ -2,5 +2,5 @@ if [ "$1" = "upgrade" ] || [ "$1" = "remove" ]; then # Return etc/cron.d/clickhouse-server to original state - service clickhouse-server disable_cron || true + service clickhouse-server disable_cron ||: fi diff --git a/debian/clickhouse-server.service b/debian/clickhouse-server.service index fa556f31951..d26a543a146 100644 --- a/debian/clickhouse-server.service +++ b/debian/clickhouse-server.service @@ -8,8 +8,6 @@ Group=clickhouse PermissionsStartOnly=true Restart=always RestartSec=30 -ExecStartPre=-/usr/bin/chown clickhouse:clickhouse -R /etc/clickhouse-server -ExecStartPre=-/bin/chown clickhouse:clickhouse -R /etc/clickhouse-server ExecStart=/usr/bin/clickhouse-server --config=/etc/clickhouse-server/config.xml LimitCORE=infinity LimitNOFILE=500000 diff --git a/debian/pbuilder-hooks/A00ccache b/debian/pbuilder-hooks/A00ccache index ab122ecf82f..53510f9d325 100755 --- a/debian/pbuilder-hooks/A00ccache +++ b/debian/pbuilder-hooks/A00ccache @@ -7,9 +7,9 @@ echo "CCACHEDIR=$CCACHEDIR CCACHE_DIR=$CCACHE_DIR SET_CCACHEDIR=$SET_CCACHEDIR" [ -z "$CCACHE_DIR" ] && export CCACHE_DIR=${CCACHEDIR:=${SET_CCACHEDIR=/var/cache/pbuilder/ccache}} if [ -n "$CCACHE_DIR" ]; then - mkdir -p $CCACHE_DIR $DISTCC_DIR || true - chown -R $BUILDUSERID:$BUILDUSERID $CCACHE_DIR $DISTCC_DIR || true - chmod -R a+rwx $CCACHE_DIR $DISTCC_DIR || true + mkdir -p $CCACHE_DIR $DISTCC_DIR ||: + chown -R $BUILDUSERID:$BUILDUSERID $CCACHE_DIR $DISTCC_DIR ||: + chmod -R a+rwx $CCACHE_DIR $DISTCC_DIR ||: fi df -h diff --git a/debian/pbuilder-hooks/B90test-server b/debian/pbuilder-hooks/B90test-server index ea1d3d78a9a..1110de53c5b 100755 --- a/debian/pbuilder-hooks/B90test-server +++ b/debian/pbuilder-hooks/B90test-server @@ -9,13 +9,13 @@ TEST_PORT_RANDOM=${TEST_PORT_RANDOM=1} if [ "${PACKAGE_INSTALL}" ]; then for PKG in $(ls /tmp/buildd/*.deb | sed -e's,.*/,,;s,_.*,,' ); do - apt-get install -y --force-yes "$PKG" || true - apt-get remove -y "$PKG" || true + apt-get install -y --force-yes "$PKG" ||: + apt-get remove -y "$PKG" ||: done - dpkg --auto-deconfigure -i /tmp/buildd/*.deb || true - apt install -y -f --allow-downgrades || true - dpkg -l | grep clickhouse || true + dpkg --auto-deconfigure -i /tmp/buildd/*.deb ||: + apt install -y -f --allow-downgrades ||: + dpkg -l | grep clickhouse ||: # Some test references uses specific timezone ln -fs /usr/share/zoneinfo/Europe/Moscow /etc/localtime @@ -49,20 +49,20 @@ if [ "${TEST_CONNECT}" ]; then echo "${CLICKHOUSE_PORT_TCP}${CLICKHOUSE_PORT_TCP_SECURE}${CLICKHOUSE_SSL_CONFIG}" > /etc/clickhouse-client/config.xml openssl dhparam -out /etc/clickhouse-server/dhparam.pem 256 openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt - chmod a+r /etc/clickhouse-server/* /etc/clickhouse-client/* + chmod a+r /etc/clickhouse-server/* /etc/clickhouse-client/* ||: CLIENT_ADD+="--secure --port ${CLICKHOUSE_PORT_TCP_SECURE}" else CLIENT_ADD+="--port ${CLICKHOUSE_PORT_TCP}" fi # For debug - # tail -n +1 -- /etc/clickhouse-server/*.xml /etc/clickhouse-server/config.d/*.xml || true + # tail -n +1 -- /etc/clickhouse-server/*.xml /etc/clickhouse-server/config.d/*.xml ||: function finish { service clickhouse-server stop - tail -n 100 /var/log/clickhouse-server/*.log || true + tail -n 100 /var/log/clickhouse-server/*.log ||: sleep 1 - killall -9 clickhouse-server || true + killall -9 clickhouse-server ||: } trap finish EXIT SIGINT SIGQUIT SIGTERM @@ -70,7 +70,7 @@ if [ "${TEST_CONNECT}" ]; then sleep ${TEST_SERVER_STARTUP_WAIT:=5} # TODO: remove me or make only on error: - tail -n100 /var/log/clickhouse-server/*.log || true + tail -n100 /var/log/clickhouse-server/*.log ||: clickhouse-client --port $CLICKHOUSE_PORT_TCP -q "SELECT * from system.build_options;" clickhouse-client ${CLIENT_ADD} -q "SELECT toDateTime(1);" diff --git a/debian/pbuilder-hooks/C99kill-make b/debian/pbuilder-hooks/C99kill-make index 863356d4724..60be8e0d402 100755 --- a/debian/pbuilder-hooks/C99kill-make +++ b/debian/pbuilder-hooks/C99kill-make @@ -2,4 +2,4 @@ # Try stop parallel build after timeout -killall make gcc gcc-7 g++-7 gcc-8 g++-8 clang clang-5.0 clang++-5.0 clang-6.0 clang++-6.0 clang-7 clang++-7 || true +killall make gcc gcc-7 g++-7 gcc-8 g++-8 clang clang-5.0 clang++-5.0 clang-6.0 clang++-6.0 clang-7 clang++-7 ||: diff --git a/libs/libdaemon/include/daemon/BaseDaemon.h b/libs/libdaemon/include/daemon/BaseDaemon.h index 65d20927322..7a16761f51c 100644 --- a/libs/libdaemon/include/daemon/BaseDaemon.h +++ b/libs/libdaemon/include/daemon/BaseDaemon.h @@ -224,7 +224,7 @@ protected: std::atomic_size_t sigint_signals_counter{0}; std::string config_path; - ConfigProcessor::LoadedConfig loaded_config; + DB::ConfigProcessor::LoadedConfig loaded_config; Poco::Util::AbstractConfiguration * last_configuration = nullptr; private: diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index f65634f1e2f..bad38c78529 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -586,7 +586,10 @@ void BaseDaemon::reloadConfiguration() * (It's convenient to log in console when you start server without any command line parameters.) */ config_path = config().getString("config-file", "config.xml"); - loaded_config = ConfigProcessor(config_path, false, true).loadConfig(/* allow_zk_includes = */ true); + DB::ConfigProcessor config_processor(config_path, false, true); + config_processor.setConfigPath(Poco::Path(config_path).makeParent().toString()); + loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true); + if (last_configuration != nullptr) config().removeConfiguration(last_configuration); last_configuration = loaded_config.configuration.duplicate(); @@ -895,7 +898,7 @@ void BaseDaemon::initialize(Application & self) umask(umask_num); } - ConfigProcessor(config_path).savePreprocessedConfig(loaded_config); + DB::ConfigProcessor(config_path).savePreprocessedConfig(loaded_config, ""); /// Write core dump on crash. { diff --git a/utils/config-processor/config-processor.cpp b/utils/config-processor/config-processor.cpp index 31eaf8522a4..242a6782b3b 100644 --- a/utils/config-processor/config-processor.cpp +++ b/utils/config-processor/config-processor.cpp @@ -11,8 +11,8 @@ int main(int argc, char ** argv) return 3; } - ConfigProcessor processor(argv[1], false, true); - XMLDocumentPtr document = processor.processConfig(); + DB::ConfigProcessor processor(argv[1], false, true); + DB::XMLDocumentPtr document = processor.processConfig(); Poco::XML::DOMWriter().writeNode(std::cout, document); } catch (Poco::Exception & e) From 65e5494326d2f22a452ee18c2e5adb3b74d363ac Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 27 Nov 2018 21:08:54 +0300 Subject: [PATCH 14/47] Fixed size for LowCardinality dictionary permutation. --- dbms/src/Columns/ColumnLowCardinality.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Columns/ColumnLowCardinality.cpp b/dbms/src/Columns/ColumnLowCardinality.cpp index da6bcfa6597..fc52d03a91d 100644 --- a/dbms/src/Columns/ColumnLowCardinality.cpp +++ b/dbms/src/Columns/ColumnLowCardinality.cpp @@ -259,7 +259,7 @@ void ColumnLowCardinality::getPermutation(bool reverse, size_t limit, int nan_di if (limit == 0) limit = size(); - size_t unique_limit = std::min(limit, getDictionary().size()); + size_t unique_limit = getDictionary().size(); Permutation unique_perm; getDictionary().getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); From 8085ce442c5412e1e712e47a163b1ab9a381607a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 27 Nov 2018 21:09:37 +0300 Subject: [PATCH 15/47] Added test for LowCardinality order by. --- .../00752_low_cardinality_permute.reference | 24 +++++++++++++++++++ .../00752_low_cardinality_permute.sql | 8 +++++++ 2 files changed, 32 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00752_low_cardinality_permute.reference create mode 100644 dbms/tests/queries/0_stateless/00752_low_cardinality_permute.sql diff --git a/dbms/tests/queries/0_stateless/00752_low_cardinality_permute.reference b/dbms/tests/queries/0_stateless/00752_low_cardinality_permute.reference new file mode 100644 index 00000000000..cd274f6acb0 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00752_low_cardinality_permute.reference @@ -0,0 +1,24 @@ +0 z +1 w +3 y +4 w +6 y +8 z +10 x +11 x +12 a +13 b +14 c +15 d +16 e +17 f +18 g +19 h +w +w +x +x +y +y +z +z diff --git a/dbms/tests/queries/0_stateless/00752_low_cardinality_permute.sql b/dbms/tests/queries/0_stateless/00752_low_cardinality_permute.sql new file mode 100644 index 00000000000..56616e5bdc9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00752_low_cardinality_permute.sql @@ -0,0 +1,8 @@ +set allow_experimental_low_cardinality_type = 1; +drop table if exists test.lc_perm; +create table test.lc_perm (val UInt32, str LowCardinality(String)) engine = MergeTree order by val; +insert into test.lc_perm values (1, 'w'), (10, 'x'), (3, 'y'), (8, 'z'), (4, 'w'), (6, 'y'), (11, 'x'), (0, 'z'), (12, 'a'), (13, 'b'), (14, 'c'), (15, 'd'), (16, 'e'), (17, 'f'), (18, 'g'), (19, 'h'); +select * from test.lc_perm; +select str from test.lc_perm where val < 12 order by str; +drop table if exists test.lc_perm; + From b1bc7481d9d6a5e01ccfce2db0eb03ec1060863b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Nov 2018 21:43:55 +0300 Subject: [PATCH 16/47] Whitespace #3603 --- dbms/src/Storages/StorageBuffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 6f850a6df30..c093641b09f 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -184,7 +184,7 @@ BlockInputStreams StorageBuffer::read( { /// Add streams to convert read blocks from the destination table. auto header = getSampleBlock(); - for (auto& stream_from_dst : streams_from_dst) + for (auto & stream_from_dst : streams_from_dst) { stream_from_dst = std::make_shared( stream_from_dst, header, getColumns().defaults, context); From cdb272848c61491ca9dc8c4bf09759bb3c0a3ff8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Nov 2018 21:44:19 +0300 Subject: [PATCH 17/47] Fixed typo in comment [#CLICKHOUSE-2] --- dbms/src/Interpreters/InterpreterDropQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index f2a3f45b55c..ae15d494814 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -89,7 +89,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t database_and_table.second->shutdown(); /// If table was already dropped by anyone, an exception will be thrown auto table_lock = database_and_table.second->lockForAlter(__PRETTY_FUNCTION__); - /// Delete table metdata and table itself from memory + /// Delete table metadata and table itself from memory database_and_table.first->removeTable(context, database_and_table.second->getTableName()); /// Delete table data database_and_table.second->drop(); From 0dfa73221c0da8c0de95a37bd7c18ec30cca78f1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Nov 2018 13:37:08 +0300 Subject: [PATCH 18/47] Removed temporary compatibility fix for Yandex.Metrika. [#CLICKHOUSE-2] --- dbms/src/Parsers/ExpressionElementParsers.cpp | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index b8007e98bdc..f3e9e43aa19 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -238,17 +238,6 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) , ErrorCodes::SYNTAX_ERROR); } - /// Temporary compatibility fix for Yandex.Metrika. - /// When we have a query with - /// cast(x, 'Type') - /// when cast is not in uppercase and when expression is written as a function, not as operator like cast(x AS Type) - /// and newer ClickHouse server (1.1.54388) interacts with older ClickHouse server (1.1.54381) in distributed query, - /// then exception was thrown. - - auto & identifier_concrete = typeid_cast(*identifier); - if (Poco::toLower(identifier_concrete.name) == "cast") - identifier_concrete.name = "CAST"; - /// The parametric aggregate function has two lists (parameters and arguments) in parentheses. Example: quantile(0.9)(x). if (pos->type == TokenType::OpeningRoundBracket) { From 6bd61d17c9f0e5edc782665e70e10faec7d40f88 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 28 Nov 2018 14:11:10 +0300 Subject: [PATCH 19/47] Disable Date and numeric types comparsion. --- dbms/src/Functions/FunctionsComparison.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index 65d6d1cbad5..f2b377490da 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -1092,7 +1092,10 @@ public: const DataTypeTuple * left_tuple = checkAndGetDataType(arguments[0].get()); const DataTypeTuple * right_tuple = checkAndGetDataType(arguments[1].get()); - if (!((arguments[0]->isValueRepresentedByNumber() && arguments[1]->isValueRepresentedByNumber()) + bool both_represented_by_number = arguments[0]->isValueRepresentedByNumber() && arguments[1]->isValueRepresentedByNumber(); + bool has_date = left.isDate() && right.isDate(); + + if (!((both_represented_by_number && !has_date) /// Do not allow compare date and number. || (left.isStringOrFixedString() && right.isStringOrFixedString()) || (left.isDate() && right.isDate()) || (left.isDate() && right.isString()) /// You can compare the date, datetime and an enumeration with a constant string. From dd2371e07110900e0f1222b5d76d96a8a10c344d Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 28 Nov 2018 14:37:12 +0300 Subject: [PATCH 20/47] CLICKHOUSE-4137 DictionaryFactory, DictionarySourceFactory (#3653) * Split ComplexKeyCacheDictionary to faster compile (part2) * Dictionaries as lib WIP * wip * clean * Fix build with old capnp * fix * wip * fixes * fix * clean * clean * clean * wip * wip * wip * flat * wip * cache * clean * wip * faster * fix style * fixes * clean * clean * Split CacheDictionary.cpp for faster compile * fix * fix * Less memory usage while compiling * missing file * format * Update registerDictionaries.h * clean --- cmake/generate_code.cmake | 5 + dbms/CMakeLists.txt | 8 +- dbms/programs/local/LocalServer.cpp | 2 + dbms/programs/server/Server.cpp | 2 + dbms/src/Dictionaries/CMakeLists.txt | 43 ++ dbms/src/Dictionaries/CacheDictionary.cpp | 494 ++---------------- dbms/src/Dictionaries/CacheDictionary.h | 6 +- dbms/src/Dictionaries/CacheDictionary.inc.h | 403 ++++++++++++++ .../CacheDictionary_generate1.cpp.in | 24 + .../CacheDictionary_generate2.cpp.in | 25 + .../CacheDictionary_generate3.cpp.in | 22 + .../ClickHouseDictionarySource.cpp | 24 +- .../Dictionaries/ClickHouseDictionarySource.h | 7 +- .../ComplexKeyCacheDictionary.cpp | 34 +- .../Dictionaries/ComplexKeyCacheDictionary.h | 6 +- ...acheDictionary_createAttributeWithType.cpp | 2 +- .../ComplexKeyCacheDictionary_generate1.cpp | 40 -- ...ComplexKeyCacheDictionary_generate1.cpp.in | 24 + .../ComplexKeyCacheDictionary_generate2.cpp | 41 -- ...ComplexKeyCacheDictionary_generate2.cpp.in | 27 + .../ComplexKeyCacheDictionary_generate3.cpp | 41 -- ...ComplexKeyCacheDictionary_generate3.cpp.in | 27 + ...exKeyCacheDictionary_setAttributeValue.cpp | 2 +- ...cheDictionary_setDefaultAttributeValue.cpp | 2 +- .../ComplexKeyHashedDictionary.cpp | 25 +- .../Dictionaries/ComplexKeyHashedDictionary.h | 7 +- .../Dictionaries/DictionaryBlockInputStream.h | 6 +- .../DictionaryBlockInputStreamBase.cpp | 2 +- dbms/src/Dictionaries/DictionaryFactory.cpp | 51 ++ dbms/src/Dictionaries/DictionaryFactory.h | 31 +- .../Dictionaries/DictionarySourceFactory.cpp | 196 ++----- .../Dictionaries/DictionarySourceFactory.h | 24 +- .../Dictionaries/DictionarySourceHelpers.cpp | 4 +- dbms/src/Dictionaries/DictionaryStructure.cpp | 2 +- dbms/src/Dictionaries/Embedded/CMakeLists.txt | 5 + .../Embedded/GeoDictionariesLoader.cpp | 8 +- .../Embedded/GeoDictionariesLoader.h | 2 +- .../Embedded/GeodataProviders/Entries.h | 2 +- .../GeodataProviders/HierarchiesProvider.cpp | 5 +- .../GeodataProviders/HierarchiesProvider.h | 3 +- .../HierarchyFormatReader.cpp | 2 +- .../GeodataProviders/HierarchyFormatReader.h | 3 +- .../GeodataProviders/IHierarchiesProvider.h | 3 +- .../GeodataProviders/INamesProvider.h | 3 +- .../GeodataProviders/NamesFormatReader.cpp | 2 +- .../GeodataProviders/NamesFormatReader.h | 3 +- .../GeodataProviders/NamesProvider.cpp | 4 +- .../Embedded/GeodataProviders/NamesProvider.h | 3 +- .../Embedded/IGeoDictionariesLoader.h | 17 +- .../Embedded/RegionsHierarchies.cpp | 3 +- .../Embedded/RegionsHierarchies.h | 6 +- .../Embedded/RegionsHierarchy.cpp | 6 +- .../Dictionaries/Embedded/RegionsHierarchy.h | 3 +- .../Dictionaries/Embedded/RegionsNames.cpp | 6 +- dbms/src/Dictionaries/Embedded/RegionsNames.h | 5 +- .../Embedded/TechDataHierarchy.cpp | 2 +- .../Dictionaries/Embedded/TechDataHierarchy.h | 14 +- .../ExecutableDictionarySource.cpp | 22 +- .../Dictionaries/ExecutableDictionarySource.h | 4 +- .../src/Dictionaries/ExternalQueryBuilder.cpp | 6 +- .../ExternalResultDescription.cpp | 2 +- .../src/Dictionaries/FileDictionarySource.cpp | 25 +- dbms/src/Dictionaries/FileDictionarySource.h | 3 +- dbms/src/Dictionaries/FlatDictionary.cpp | 36 +- dbms/src/Dictionaries/FlatDictionary.h | 6 +- .../src/Dictionaries/HTTPDictionarySource.cpp | 22 +- dbms/src/Dictionaries/HTTPDictionarySource.h | 4 +- dbms/src/Dictionaries/HashedDictionary.cpp | 31 +- dbms/src/Dictionaries/HashedDictionary.h | 6 +- dbms/src/Dictionaries/IDictionary.h | 2 +- .../Dictionaries/LibraryDictionarySource.cpp | 20 +- .../Dictionaries/LibraryDictionarySource.h | 7 +- .../LibraryDictionarySourceExternal.cpp | 2 +- .../Dictionaries/MongoDBBlockInputStream.cpp | 4 +- .../Dictionaries/MongoDBBlockInputStream.h | 2 +- .../Dictionaries/MongoDBDictionarySource.cpp | 45 +- .../Dictionaries/MongoDBDictionarySource.h | 13 +- .../Dictionaries/MySQLBlockInputStream.cpp | 2 +- dbms/src/Dictionaries/MySQLBlockInputStream.h | 2 +- .../Dictionaries/MySQLDictionarySource.cpp | 51 +- dbms/src/Dictionaries/MySQLDictionarySource.h | 11 +- .../src/Dictionaries/ODBCBlockInputStream.cpp | 5 +- dbms/src/Dictionaries/ODBCBlockInputStream.h | 4 +- .../RangeDictionaryBlockInputStream.h | 8 +- .../Dictionaries/RangeHashedDictionary.cpp | 31 +- dbms/src/Dictionaries/RangeHashedDictionary.h | 6 +- dbms/src/Dictionaries/TrieDictionary.cpp | 26 +- dbms/src/Dictionaries/TrieDictionary.h | 7 +- .../src/Dictionaries/XDBCDictionarySource.cpp | 54 +- dbms/src/Dictionaries/XDBCDictionarySource.h | 9 +- dbms/src/Dictionaries/readInvalidateQuery.cpp | 2 +- .../src/Dictionaries/registerDictionaries.cpp | 52 ++ dbms/src/Dictionaries/registerDictionaries.h | 6 + .../Dictionaries/writeParenthesisedString.cpp | 2 +- dbms/src/Functions/CMakeLists.txt | 13 +- dbms/src/IO/HTTPCommon.cpp | 2 + .../CatBoostModel.cpp | 3 +- .../CatBoostModel.h | 0 dbms/src/Interpreters/DictionaryFactory.cpp | 133 ----- dbms/src/Interpreters/ExternalModels.h | 2 +- .../Storages/System/StorageSystemModels.cpp | 2 +- 101 files changed, 1407 insertions(+), 1057 deletions(-) create mode 100644 cmake/generate_code.cmake create mode 100644 dbms/src/Dictionaries/CacheDictionary.inc.h create mode 100644 dbms/src/Dictionaries/CacheDictionary_generate1.cpp.in create mode 100644 dbms/src/Dictionaries/CacheDictionary_generate2.cpp.in create mode 100644 dbms/src/Dictionaries/CacheDictionary_generate3.cpp.in delete mode 100644 dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp create mode 100644 dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp.in delete mode 100644 dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp create mode 100644 dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp.in delete mode 100644 dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp create mode 100644 dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp.in create mode 100644 dbms/src/Dictionaries/DictionaryFactory.cpp create mode 100644 dbms/src/Dictionaries/Embedded/CMakeLists.txt create mode 100644 dbms/src/Dictionaries/registerDictionaries.cpp create mode 100644 dbms/src/Dictionaries/registerDictionaries.h rename dbms/src/{Dictionaries => Interpreters}/CatBoostModel.cpp (99%) rename dbms/src/{Dictionaries => Interpreters}/CatBoostModel.h (100%) delete mode 100644 dbms/src/Interpreters/DictionaryFactory.cpp diff --git a/cmake/generate_code.cmake b/cmake/generate_code.cmake new file mode 100644 index 00000000000..8eb9da24d1d --- /dev/null +++ b/cmake/generate_code.cmake @@ -0,0 +1,5 @@ +function(generate_code TEMPLATE_FILE) + foreach(NAME IN LISTS ARGN) + configure_file (${TEMPLATE_FILE}.cpp.in ${CMAKE_CURRENT_BINARY_DIR}/generated/${TEMPLATE_FILE}_${NAME}.cpp) + endforeach() +endfunction() diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index f0116d768e5..cce97e4a57e 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -63,9 +63,6 @@ add_headers_and_sources(dbms src/Core) add_headers_and_sources(dbms src/DataStreams) add_headers_and_sources(dbms src/DataTypes) add_headers_and_sources(dbms src/Databases) -add_headers_and_sources(dbms src/Dictionaries) -add_headers_and_sources(dbms src/Dictionaries/Embedded) -add_headers_and_sources(dbms src/Dictionaries/Embedded/GeodataProviders) add_headers_and_sources(dbms src/Interpreters) add_headers_and_sources(dbms src/Interpreters/ClusterProxy) add_headers_and_sources(dbms src/Columns) @@ -184,8 +181,11 @@ target_link_libraries (dbms clickhouse_common_config PUBLIC clickhouse_common_io - pocoext + PRIVATE + clickhouse_dictionaries + clickhouse_dictionaries_embedded PUBLIC + pocoext ${MYSQLXX_LIBRARY} PRIVATE ${BTRIE_LIBRARIES} diff --git a/dbms/programs/local/LocalServer.cpp b/dbms/programs/local/LocalServer.cpp index 0dab224c7f1..40e26438afc 100644 --- a/dbms/programs/local/LocalServer.cpp +++ b/dbms/programs/local/LocalServer.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include @@ -142,6 +143,7 @@ try registerAggregateFunctions(); registerTableFunctions(); registerStorages(); + registerDictionaries(); /// Maybe useless if (config().has("macros")) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 8d3bff9ebfb..32f88079ce3 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -36,6 +36,7 @@ #include #include #include +#include #include #include "HTTPHandlerFactory.h" #include "MetricsTransmitter.h" @@ -109,6 +110,7 @@ int Server::main(const std::vector & /*args*/) registerAggregateFunctions(); registerTableFunctions(); registerStorages(); + registerDictionaries(); CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::get()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); diff --git a/dbms/src/Dictionaries/CMakeLists.txt b/dbms/src/Dictionaries/CMakeLists.txt index e69de29bb2d..08624bd6c6a 100644 --- a/dbms/src/Dictionaries/CMakeLists.txt +++ b/dbms/src/Dictionaries/CMakeLists.txt @@ -0,0 +1,43 @@ +include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) +include(${ClickHouse_SOURCE_DIR}/cmake/generate_code.cmake) + +add_headers_and_sources(clickhouse_dictionaries .) + +generate_code(ComplexKeyCacheDictionary_generate1 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128) +generate_code(ComplexKeyCacheDictionary_generate2 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128) +generate_code(ComplexKeyCacheDictionary_generate3 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128) +generate_code(CacheDictionary_generate1 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128) +generate_code(CacheDictionary_generate2 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128) +generate_code(CacheDictionary_generate3 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128) +add_headers_and_sources(clickhouse_dictionaries ${CMAKE_CURRENT_BINARY_DIR}/generated/) + +add_library(clickhouse_dictionaries ${LINK_MODE} ${clickhouse_dictionaries_sources}) +target_link_libraries(clickhouse_dictionaries PRIVATE clickhouse_common_io pocoext ${MYSQLXX_LIBRARY} ${BTRIE_LIBRARIES}) + +if(Poco_SQL_FOUND AND NOT USE_INTERNAL_POCO_LIBRARY) + target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${Poco_SQL_INCLUDE_DIR}) +endif() + +if(USE_POCO_SQLODBC) + target_link_libraries(clickhouse_dictionaries PRIVATE ${Poco_SQLODBC_LIBRARY} ${Poco_SQL_LIBRARY}) + if (NOT USE_INTERNAL_POCO_LIBRARY) + target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_SQLODBC_INCLUDE_DIR} ${Poco_SQL_INCLUDE_DIR}) + endif() +endif() + +if(Poco_Data_FOUND) + target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${Poco_Data_INCLUDE_DIR}) +endif() + +if(USE_POCO_DATAODBC) + target_link_libraries(clickhouse_dictionaries PRIVATE ${Poco_DataODBC_LIBRARY} ${Poco_Data_LIBRARY}) + if (NOT USE_INTERNAL_POCO_LIBRARY) + target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_DataODBC_INCLUDE_DIR}) + endif() +endif() + +if(USE_POCO_MONGODB) + target_link_libraries(clickhouse_dictionaries PRIVATE ${Poco_MongoDB_LIBRARY}) +endif() + +add_subdirectory(Embedded) diff --git a/dbms/src/Dictionaries/CacheDictionary.cpp b/dbms/src/Dictionaries/CacheDictionary.cpp index 41950b443f1..08a0752a23f 100644 --- a/dbms/src/Dictionaries/CacheDictionary.cpp +++ b/dbms/src/Dictionaries/CacheDictionary.cpp @@ -1,3 +1,5 @@ +#include "CacheDictionary.h" + #include #include #include @@ -11,12 +13,12 @@ #include #include #include -#include -#include +#include "DictionaryBlockInputStream.h" #include #include #include - +#include "DictionaryFactory.h" +#include "CacheDictionary.inc.h" namespace ProfileEvents { @@ -47,6 +49,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int UNSUPPORTED_METHOD; extern const int LOGICAL_ERROR; + extern const int TOO_SMALL_BUFFER_SIZE; } @@ -206,34 +209,6 @@ void CacheDictionary::isInConstantVector( out[i] = std::find(ancestors.begin(), ancestors.end(), ancestor_ids[i]) != ancestors.end(); } - -#define DECLARE(TYPE)\ -void CacheDictionary::get##TYPE(const std::string & attribute_name, const PaddedPODArray & ids, ResultArrayType & out) const\ -{\ - auto & attribute = getAttribute(attribute_name);\ - if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE))\ - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH};\ - \ - const auto null_value = std::get(attribute.null_values);\ - \ - getItemsNumber(attribute, ids, out, [&] (const size_t) { return null_value; });\ -} -DECLARE(UInt8) -DECLARE(UInt16) -DECLARE(UInt32) -DECLARE(UInt64) -DECLARE(UInt128) -DECLARE(Int8) -DECLARE(Int16) -DECLARE(Int32) -DECLARE(Int64) -DECLARE(Float32) -DECLARE(Float64) -DECLARE(Decimal32) -DECLARE(Decimal64) -DECLARE(Decimal128) -#undef DECLARE - void CacheDictionary::getString(const std::string & attribute_name, const PaddedPODArray & ids, ColumnString * out) const { auto & attribute = getAttribute(attribute_name); @@ -245,33 +220,6 @@ void CacheDictionary::getString(const std::string & attribute_name, const Padded getItemsString(attribute, ids, out, [&] (const size_t) { return null_value; }); } -#define DECLARE(TYPE)\ -void CacheDictionary::get##TYPE(\ - const std::string & attribute_name, const PaddedPODArray & ids, const PaddedPODArray & def,\ - ResultArrayType & out) const\ -{\ - auto & attribute = getAttribute(attribute_name);\ - if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE))\ - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH};\ - \ - getItemsNumber(attribute, ids, out, [&] (const size_t row) { return def[row]; });\ -} -DECLARE(UInt8) -DECLARE(UInt16) -DECLARE(UInt32) -DECLARE(UInt64) -DECLARE(UInt128) -DECLARE(Int8) -DECLARE(Int16) -DECLARE(Int32) -DECLARE(Int64) -DECLARE(Float32) -DECLARE(Float64) -DECLARE(Decimal32) -DECLARE(Decimal64) -DECLARE(Decimal128) -#undef DECLARE - void CacheDictionary::getString( const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, ColumnString * const out) const @@ -283,32 +231,6 @@ void CacheDictionary::getString( getItemsString(attribute, ids, out, [&] (const size_t row) { return def->getDataAt(row); }); } -#define DECLARE(TYPE)\ -void CacheDictionary::get##TYPE(\ - const std::string & attribute_name, const PaddedPODArray & ids, const TYPE def, ResultArrayType & out) const\ -{\ - auto & attribute = getAttribute(attribute_name);\ - if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE))\ - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH};\ - \ - getItemsNumber(attribute, ids, out, [&] (const size_t) { return def; });\ -} -DECLARE(UInt8) -DECLARE(UInt16) -DECLARE(UInt32) -DECLARE(UInt64) -DECLARE(UInt128) -DECLARE(Int8) -DECLARE(Int16) -DECLARE(Int32) -DECLARE(Int64) -DECLARE(Float32) -DECLARE(Float64) -DECLARE(Decimal32) -DECLARE(Decimal64) -DECLARE(Decimal128) -#undef DECLARE - void CacheDictionary::getString( const std::string & attribute_name, const PaddedPODArray & ids, const String & def, ColumnString * const out) const @@ -487,374 +409,6 @@ CacheDictionary::Attribute CacheDictionary::createAttributeWithType(const Attrib return attr; } - -template -void CacheDictionary::getItemsNumber( - Attribute & attribute, - const PaddedPODArray & ids, - ResultArrayType & out, - DefaultGetter && get_default) const -{ - if (false) {} -#define DISPATCH(TYPE) \ - else if (attribute.type == AttributeUnderlyingType::TYPE) \ - getItemsNumberImpl(attribute, ids, out, std::forward(get_default)); - DISPATCH(UInt8) - DISPATCH(UInt16) - DISPATCH(UInt32) - DISPATCH(UInt64) - DISPATCH(UInt128) - DISPATCH(Int8) - DISPATCH(Int16) - DISPATCH(Int32) - DISPATCH(Int64) - DISPATCH(Float32) - DISPATCH(Float64) - DISPATCH(Decimal32) - DISPATCH(Decimal64) - DISPATCH(Decimal128) -#undef DISPATCH - else - throw Exception("Unexpected type of attribute: " + toString(attribute.type), ErrorCodes::LOGICAL_ERROR); -} - -template -void CacheDictionary::getItemsNumberImpl( - Attribute & attribute, - const PaddedPODArray & ids, - ResultArrayType & out, - DefaultGetter && get_default) const -{ - /// Mapping: -> { all indices `i` of `ids` such that `ids[i]` = } - std::unordered_map> outdated_ids; - auto & attribute_array = std::get>(attribute.arrays); - const auto rows = ext::size(ids); - - size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0; - - { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - - const auto now = std::chrono::system_clock::now(); - /// fetch up-to-date values, decide which ones require update - for (const auto row : ext::range(0, rows)) - { - const auto id = ids[row]; - - /** cell should be updated if either: - * 1. ids do not match, - * 2. cell has expired, - * 3. explicit defaults were specified and cell was set default. */ - - const auto find_result = findCellIdx(id, now); - if (!find_result.valid) - { - outdated_ids[id].push_back(row); - if (find_result.outdated) - ++cache_expired; - else - ++cache_not_found; - } - else - { - ++cache_hit; - const auto & cell_idx = find_result.cell_idx; - const auto & cell = cells[cell_idx]; - out[row] = cell.isDefault() ? get_default(row) : static_cast(attribute_array[cell_idx]); - } - } - } - - ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); - ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); - ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); - - query_count.fetch_add(rows, std::memory_order_relaxed); - hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release); - - if (outdated_ids.empty()) - return; - - std::vector required_ids(outdated_ids.size()); - std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), - [] (auto & pair) { return pair.first; }); - - /// request new values - update(required_ids, - [&] (const auto id, const auto cell_idx) - { - const auto attribute_value = attribute_array[cell_idx]; - - for (const size_t row : outdated_ids[id]) - out[row] = static_cast(attribute_value); - }, - [&] (const auto id, const auto) - { - for (const size_t row : outdated_ids[id]) - out[row] = get_default(row); - }); -} - -template -void CacheDictionary::getItemsString( - Attribute & attribute, - const PaddedPODArray & ids, - ColumnString * out, - DefaultGetter && get_default) const -{ - const auto rows = ext::size(ids); - - /// save on some allocations - out->getOffsets().reserve(rows); - - auto & attribute_array = std::get>(attribute.arrays); - - auto found_outdated_values = false; - - /// perform optimistic version, fallback to pessimistic if failed - { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - - const auto now = std::chrono::system_clock::now(); - /// fetch up-to-date values, discard on fail - for (const auto row : ext::range(0, rows)) - { - const auto id = ids[row]; - - const auto find_result = findCellIdx(id, now); - if (!find_result.valid) - { - found_outdated_values = true; - break; - } - else - { - const auto & cell_idx = find_result.cell_idx; - const auto & cell = cells[cell_idx]; - const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx]; - out->insertData(string_ref.data, string_ref.size); - } - } - } - - /// optimistic code completed successfully - if (!found_outdated_values) - { - query_count.fetch_add(rows, std::memory_order_relaxed); - hit_count.fetch_add(rows, std::memory_order_release); - return; - } - - /// now onto the pessimistic one, discard possible partial results from the optimistic path - out->getChars().resize_assume_reserved(0); - out->getOffsets().resize_assume_reserved(0); - - /// Mapping: -> { all indices `i` of `ids` such that `ids[i]` = } - std::unordered_map> outdated_ids; - /// we are going to store every string separately - std::unordered_map map; - - size_t total_length = 0; - size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0; - { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - - const auto now = std::chrono::system_clock::now(); - for (const auto row : ext::range(0, ids.size())) - { - const auto id = ids[row]; - - const auto find_result = findCellIdx(id, now); - if (!find_result.valid) - { - outdated_ids[id].push_back(row); - if (find_result.outdated) - ++cache_expired; - else - ++cache_not_found; - } - else - { - ++cache_hit; - const auto & cell_idx = find_result.cell_idx; - const auto & cell = cells[cell_idx]; - const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx]; - - if (!cell.isDefault()) - map[id] = String{string_ref}; - - total_length += string_ref.size + 1; - } - } - } - - ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); - ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); - ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); - - query_count.fetch_add(rows, std::memory_order_relaxed); - hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release); - - /// request new values - if (!outdated_ids.empty()) - { - std::vector required_ids(outdated_ids.size()); - std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), - [] (auto & pair) { return pair.first; }); - - update(required_ids, - [&] (const auto id, const auto cell_idx) - { - const auto attribute_value = attribute_array[cell_idx]; - - map[id] = String{attribute_value}; - total_length += (attribute_value.size + 1) * outdated_ids[id].size(); - }, - [&] (const auto id, const auto) - { - for (const auto row : outdated_ids[id]) - total_length += get_default(row).size + 1; - }); - } - - out->getChars().reserve(total_length); - - for (const auto row : ext::range(0, ext::size(ids))) - { - const auto id = ids[row]; - const auto it = map.find(id); - - const auto string_ref = it != std::end(map) ? StringRef{it->second} : get_default(row); - out->insertData(string_ref.data, string_ref.size); - } -} - -template -void CacheDictionary::update( - const std::vector & requested_ids, - PresentIdHandler && on_cell_updated, - AbsentIdHandler && on_id_not_found) const -{ - std::unordered_map remaining_ids{requested_ids.size()}; - for (const auto id : requested_ids) - remaining_ids.insert({ id, 0 }); - - std::uniform_int_distribution distribution - { - dict_lifetime.min_sec, - dict_lifetime.max_sec - }; - - const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; - - { - CurrentMetrics::Increment metric_increment{CurrentMetrics::DictCacheRequests}; - Stopwatch watch; - auto stream = source_ptr->loadIds(requested_ids); - stream->readPrefix(); - - const auto now = std::chrono::system_clock::now(); - - while (const auto block = stream->read()) - { - const auto id_column = typeid_cast(block.safeGetByPosition(0).column.get()); - if (!id_column) - throw Exception{name + ": id column has type different from UInt64.", ErrorCodes::TYPE_MISMATCH}; - - const auto & ids = id_column->getData(); - - /// cache column pointers - const auto column_ptrs = ext::map(ext::range(0, attributes.size()), [&block] (size_t i) - { - return block.safeGetByPosition(i + 1).column.get(); - }); - - for (const auto i : ext::range(0, ids.size())) - { - const auto id = ids[i]; - - const auto find_result = findCellIdx(id, now); - const auto & cell_idx = find_result.cell_idx; - - auto & cell = cells[cell_idx]; - - for (const auto attribute_idx : ext::range(0, attributes.size())) - { - const auto & attribute_column = *column_ptrs[attribute_idx]; - auto & attribute = attributes[attribute_idx]; - - setAttributeValue(attribute, cell_idx, attribute_column[i]); - } - - /// if cell id is zero and zero does not map to this cell, then the cell is unused - if (cell.id == 0 && cell_idx != zero_cell_idx) - element_count.fetch_add(1, std::memory_order_relaxed); - - cell.id = id; - if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0) - cell.setExpiresAt(std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)}); - else - cell.setExpiresAt(std::chrono::time_point::max()); - - /// inform caller - on_cell_updated(id, cell_idx); - /// mark corresponding id as found - remaining_ids[id] = 1; - } - } - - stream->readSuffix(); - - ProfileEvents::increment(ProfileEvents::DictCacheKeysRequested, requested_ids.size()); - ProfileEvents::increment(ProfileEvents::DictCacheRequestTimeNs, watch.elapsed()); - } - - size_t not_found_num = 0, found_num = 0; - - const auto now = std::chrono::system_clock::now(); - /// Check which ids have not been found and require setting null_value - for (const auto & id_found_pair : remaining_ids) - { - if (id_found_pair.second) - { - ++found_num; - continue; - } - ++not_found_num; - - const auto id = id_found_pair.first; - - const auto find_result = findCellIdx(id, now); - const auto & cell_idx = find_result.cell_idx; - - auto & cell = cells[cell_idx]; - - /// Set null_value for each attribute - for (auto & attribute : attributes) - setDefaultAttributeValue(attribute, cell_idx); - - /// Check if cell had not been occupied before and increment element counter if it hadn't - if (cell.id == 0 && cell_idx != zero_cell_idx) - element_count.fetch_add(1, std::memory_order_relaxed); - - cell.id = id; - if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0) - cell.setExpiresAt(std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)}); - else - cell.setExpiresAt(std::chrono::time_point::max()); - - cell.setDefault(); - - /// inform caller that the cell has not been found - on_id_not_found(id, cell_idx); - } - - ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedMiss, not_found_num); - ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedFound, found_num); - ProfileEvents::increment(ProfileEvents::DictCacheRequests); -} - - void CacheDictionary::setDefaultAttributeValue(Attribute & attribute, const Key idx) const { switch (attribute.type) @@ -981,5 +535,41 @@ BlockInputStreamPtr CacheDictionary::getBlockInputStream(const Names & column_na return std::make_shared(shared_from_this(), max_block_size, getCachedIds(), column_names); } +void registerDictionaryCache(DictionaryFactory & factory) +{ + auto create_layout = [=]( + const std::string & name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr + ) -> DictionaryPtr { + + if (dict_struct.key) + throw Exception {"'key' is not supported for dictionary of layout 'cache'", ErrorCodes::UNSUPPORTED_METHOD}; + + if (dict_struct.range_min || dict_struct.range_max) + throw Exception {name + + ": elements .structure.range_min and .structure.range_max should be defined only " + "for a dictionary of layout 'range_hashed'", + ErrorCodes::BAD_ARGUMENTS}; + const auto & layout_prefix = config_prefix + ".layout"; + const auto size = config.getInt(layout_prefix + ".cache.size_in_cells"); + if (size == 0) + throw Exception {name + ": dictionary of layout 'cache' cannot have 0 cells", ErrorCodes::TOO_SMALL_BUFFER_SIZE}; + + const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); + if (require_nonempty) + throw Exception {name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set", + ErrorCodes::BAD_ARGUMENTS}; + + const DictionaryLifetime dict_lifetime {config, config_prefix + ".lifetime"}; + return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, size); + + + }; + factory.registerLayout("cache", create_layout); +} + } diff --git a/dbms/src/Dictionaries/CacheDictionary.h b/dbms/src/Dictionaries/CacheDictionary.h index 453e38246ec..8b72daaca23 100644 --- a/dbms/src/Dictionaries/CacheDictionary.h +++ b/dbms/src/Dictionaries/CacheDictionary.h @@ -1,8 +1,8 @@ #pragma once -#include -#include -#include +#include "IDictionary.h" +#include "IDictionarySource.h" +#include "DictionaryStructure.h" #include #include #include diff --git a/dbms/src/Dictionaries/CacheDictionary.inc.h b/dbms/src/Dictionaries/CacheDictionary.inc.h new file mode 100644 index 00000000000..6fc082ab267 --- /dev/null +++ b/dbms/src/Dictionaries/CacheDictionary.inc.h @@ -0,0 +1,403 @@ +#include "CacheDictionary.h" + +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event DictCacheKeysRequested; + extern const Event DictCacheKeysRequestedMiss; + extern const Event DictCacheKeysRequestedFound; + extern const Event DictCacheKeysExpired; + extern const Event DictCacheKeysNotFound; + extern const Event DictCacheKeysHit; + extern const Event DictCacheRequestTimeNs; + extern const Event DictCacheRequests; + extern const Event DictCacheLockWriteNs; + extern const Event DictCacheLockReadNs; +} + +namespace CurrentMetrics +{ + extern const Metric DictCacheRequests; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; +} + +template +void CacheDictionary::getItemsNumber( + Attribute & attribute, + const PaddedPODArray & ids, + ResultArrayType & out, + DefaultGetter && get_default) const +{ + if (false) {} +#define DISPATCH(TYPE) \ + else if (attribute.type == AttributeUnderlyingType::TYPE) \ + getItemsNumberImpl(attribute, ids, out, std::forward(get_default)); + DISPATCH(UInt8) + DISPATCH(UInt16) + DISPATCH(UInt32) + DISPATCH(UInt64) + DISPATCH(UInt128) + DISPATCH(Int8) + DISPATCH(Int16) + DISPATCH(Int32) + DISPATCH(Int64) + DISPATCH(Float32) + DISPATCH(Float64) + DISPATCH(Decimal32) + DISPATCH(Decimal64) + DISPATCH(Decimal128) +#undef DISPATCH + else + throw Exception("Unexpected type of attribute: " + toString(attribute.type), ErrorCodes::LOGICAL_ERROR); +} + +template +void CacheDictionary::getItemsNumberImpl( + Attribute & attribute, + const PaddedPODArray & ids, + ResultArrayType & out, + DefaultGetter && get_default) const +{ + /// Mapping: -> { all indices `i` of `ids` such that `ids[i]` = } + std::unordered_map> outdated_ids; + auto & attribute_array = std::get>(attribute.arrays); + const auto rows = ext::size(ids); + + size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0; + + { + const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; + + const auto now = std::chrono::system_clock::now(); + /// fetch up-to-date values, decide which ones require update + for (const auto row : ext::range(0, rows)) + { + const auto id = ids[row]; + + /** cell should be updated if either: + * 1. ids do not match, + * 2. cell has expired, + * 3. explicit defaults were specified and cell was set default. */ + + const auto find_result = findCellIdx(id, now); + if (!find_result.valid) + { + outdated_ids[id].push_back(row); + if (find_result.outdated) + ++cache_expired; + else + ++cache_not_found; + } + else + { + ++cache_hit; + const auto & cell_idx = find_result.cell_idx; + const auto & cell = cells[cell_idx]; + out[row] = cell.isDefault() ? get_default(row) : static_cast(attribute_array[cell_idx]); + } + } + } + + ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); + ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); + ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); + + query_count.fetch_add(rows, std::memory_order_relaxed); + hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release); + + if (outdated_ids.empty()) + return; + + std::vector required_ids(outdated_ids.size()); + std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), + [] (auto & pair) { return pair.first; }); + + /// request new values + update(required_ids, + [&] (const auto id, const auto cell_idx) + { + const auto attribute_value = attribute_array[cell_idx]; + + for (const size_t row : outdated_ids[id]) + out[row] = static_cast(attribute_value); + }, + [&] (const auto id, const auto) + { + for (const size_t row : outdated_ids[id]) + out[row] = get_default(row); + }); +} + +template +void CacheDictionary::getItemsString( + Attribute & attribute, + const PaddedPODArray & ids, + ColumnString * out, + DefaultGetter && get_default) const +{ + const auto rows = ext::size(ids); + + /// save on some allocations + out->getOffsets().reserve(rows); + + auto & attribute_array = std::get>(attribute.arrays); + + auto found_outdated_values = false; + + /// perform optimistic version, fallback to pessimistic if failed + { + const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; + + const auto now = std::chrono::system_clock::now(); + /// fetch up-to-date values, discard on fail + for (const auto row : ext::range(0, rows)) + { + const auto id = ids[row]; + + const auto find_result = findCellIdx(id, now); + if (!find_result.valid) + { + found_outdated_values = true; + break; + } + else + { + const auto & cell_idx = find_result.cell_idx; + const auto & cell = cells[cell_idx]; + const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx]; + out->insertData(string_ref.data, string_ref.size); + } + } + } + + /// optimistic code completed successfully + if (!found_outdated_values) + { + query_count.fetch_add(rows, std::memory_order_relaxed); + hit_count.fetch_add(rows, std::memory_order_release); + return; + } + + /// now onto the pessimistic one, discard possible partial results from the optimistic path + out->getChars().resize_assume_reserved(0); + out->getOffsets().resize_assume_reserved(0); + + /// Mapping: -> { all indices `i` of `ids` such that `ids[i]` = } + std::unordered_map> outdated_ids; + /// we are going to store every string separately + std::unordered_map map; + + size_t total_length = 0; + size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0; + { + const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; + + const auto now = std::chrono::system_clock::now(); + for (const auto row : ext::range(0, ids.size())) + { + const auto id = ids[row]; + + const auto find_result = findCellIdx(id, now); + if (!find_result.valid) + { + outdated_ids[id].push_back(row); + if (find_result.outdated) + ++cache_expired; + else + ++cache_not_found; + } + else + { + ++cache_hit; + const auto & cell_idx = find_result.cell_idx; + const auto & cell = cells[cell_idx]; + const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx]; + + if (!cell.isDefault()) + map[id] = String{string_ref}; + + total_length += string_ref.size + 1; + } + } + } + + ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); + ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); + ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); + + query_count.fetch_add(rows, std::memory_order_relaxed); + hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release); + + /// request new values + if (!outdated_ids.empty()) + { + std::vector required_ids(outdated_ids.size()); + std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), + [] (auto & pair) { return pair.first; }); + + update(required_ids, + [&] (const auto id, const auto cell_idx) + { + const auto attribute_value = attribute_array[cell_idx]; + + map[id] = String{attribute_value}; + total_length += (attribute_value.size + 1) * outdated_ids[id].size(); + }, + [&] (const auto id, const auto) + { + for (const auto row : outdated_ids[id]) + total_length += get_default(row).size + 1; + }); + } + + out->getChars().reserve(total_length); + + for (const auto row : ext::range(0, ext::size(ids))) + { + const auto id = ids[row]; + const auto it = map.find(id); + + const auto string_ref = it != std::end(map) ? StringRef{it->second} : get_default(row); + out->insertData(string_ref.data, string_ref.size); + } +} + +template +void CacheDictionary::update( + const std::vector & requested_ids, + PresentIdHandler && on_cell_updated, + AbsentIdHandler && on_id_not_found) const +{ + std::unordered_map remaining_ids{requested_ids.size()}; + for (const auto id : requested_ids) + remaining_ids.insert({ id, 0 }); + + std::uniform_int_distribution distribution + { + dict_lifetime.min_sec, + dict_lifetime.max_sec + }; + + const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; + + { + CurrentMetrics::Increment metric_increment{CurrentMetrics::DictCacheRequests}; + Stopwatch watch; + auto stream = source_ptr->loadIds(requested_ids); + stream->readPrefix(); + + const auto now = std::chrono::system_clock::now(); + + while (const auto block = stream->read()) + { + const auto id_column = typeid_cast(block.safeGetByPosition(0).column.get()); + if (!id_column) + throw Exception{name + ": id column has type different from UInt64.", ErrorCodes::TYPE_MISMATCH}; + + const auto & ids = id_column->getData(); + + /// cache column pointers + const auto column_ptrs = ext::map(ext::range(0, attributes.size()), [&block] (size_t i) + { + return block.safeGetByPosition(i + 1).column.get(); + }); + + for (const auto i : ext::range(0, ids.size())) + { + const auto id = ids[i]; + + const auto find_result = findCellIdx(id, now); + const auto & cell_idx = find_result.cell_idx; + + auto & cell = cells[cell_idx]; + + for (const auto attribute_idx : ext::range(0, attributes.size())) + { + const auto & attribute_column = *column_ptrs[attribute_idx]; + auto & attribute = attributes[attribute_idx]; + + setAttributeValue(attribute, cell_idx, attribute_column[i]); + } + + /// if cell id is zero and zero does not map to this cell, then the cell is unused + if (cell.id == 0 && cell_idx != zero_cell_idx) + element_count.fetch_add(1, std::memory_order_relaxed); + + cell.id = id; + if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0) + cell.setExpiresAt(std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)}); + else + cell.setExpiresAt(std::chrono::time_point::max()); + + /// inform caller + on_cell_updated(id, cell_idx); + /// mark corresponding id as found + remaining_ids[id] = 1; + } + } + + stream->readSuffix(); + + ProfileEvents::increment(ProfileEvents::DictCacheKeysRequested, requested_ids.size()); + ProfileEvents::increment(ProfileEvents::DictCacheRequestTimeNs, watch.elapsed()); + } + + size_t not_found_num = 0, found_num = 0; + + const auto now = std::chrono::system_clock::now(); + /// Check which ids have not been found and require setting null_value + for (const auto & id_found_pair : remaining_ids) + { + if (id_found_pair.second) + { + ++found_num; + continue; + } + ++not_found_num; + + const auto id = id_found_pair.first; + + const auto find_result = findCellIdx(id, now); + const auto & cell_idx = find_result.cell_idx; + + auto & cell = cells[cell_idx]; + + /// Set null_value for each attribute + for (auto & attribute : attributes) + setDefaultAttributeValue(attribute, cell_idx); + + /// Check if cell had not been occupied before and increment element counter if it hadn't + if (cell.id == 0 && cell_idx != zero_cell_idx) + element_count.fetch_add(1, std::memory_order_relaxed); + + cell.id = id; + if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0) + cell.setExpiresAt(std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)}); + else + cell.setExpiresAt(std::chrono::time_point::max()); + + cell.setDefault(); + + /// inform caller that the cell has not been found + on_id_not_found(id, cell_idx); + } + + ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedMiss, not_found_num); + ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedFound, found_num); + ProfileEvents::increment(ProfileEvents::DictCacheRequests); +} + +} diff --git a/dbms/src/Dictionaries/CacheDictionary_generate1.cpp.in b/dbms/src/Dictionaries/CacheDictionary_generate1.cpp.in new file mode 100644 index 00000000000..53addbed3f6 --- /dev/null +++ b/dbms/src/Dictionaries/CacheDictionary_generate1.cpp.in @@ -0,0 +1,24 @@ +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; +} + +using TYPE = @NAME@; +void CacheDictionary::get@NAME@(const std::string & attribute_name, const PaddedPODArray & ids, ResultArrayType & out) const +{ + auto & attribute = getAttribute(attribute_name); + if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::@NAME@)) + throw Exception {name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), + ErrorCodes::TYPE_MISMATCH}; + + const auto null_value = std::get(attribute.null_values); + + getItemsNumber(attribute, ids, out, [&](const size_t) { return null_value; }); +} + +} diff --git a/dbms/src/Dictionaries/CacheDictionary_generate2.cpp.in b/dbms/src/Dictionaries/CacheDictionary_generate2.cpp.in new file mode 100644 index 00000000000..5d87310030f --- /dev/null +++ b/dbms/src/Dictionaries/CacheDictionary_generate2.cpp.in @@ -0,0 +1,25 @@ +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; +} + +using TYPE = @NAME@; +void CacheDictionary::get@NAME@(const std::string & attribute_name, + const PaddedPODArray & ids, + const PaddedPODArray & def, + ResultArrayType & out) const +{ + auto & attribute = getAttribute(attribute_name); + if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::@NAME@)) + throw Exception {name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), + ErrorCodes::TYPE_MISMATCH}; + + getItemsNumber(attribute, ids, out, [&](const size_t row) { return def[row]; }); +} + +} diff --git a/dbms/src/Dictionaries/CacheDictionary_generate3.cpp.in b/dbms/src/Dictionaries/CacheDictionary_generate3.cpp.in new file mode 100644 index 00000000000..7931630d2e9 --- /dev/null +++ b/dbms/src/Dictionaries/CacheDictionary_generate3.cpp.in @@ -0,0 +1,22 @@ +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; +} + +using TYPE = @NAME@; +void CacheDictionary::get@NAME@(const std::string & attribute_name, const PaddedPODArray & ids, const TYPE def, ResultArrayType & out) const +{ + auto & attribute = getAttribute(attribute_name); + if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::@NAME@)) + throw Exception {name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), + ErrorCodes::TYPE_MISMATCH}; + + getItemsNumber(attribute, ids, out, [&](const size_t) { return def; }); +} + +} diff --git a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp index 4a52f4a9f4f..161a157ffaa 100644 --- a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -1,14 +1,17 @@ -#include -#include -#include +#include "ClickHouseDictionarySource.h" +#include "ExternalQueryBuilder.h" +#include "writeParenthesisedString.h" #include #include -#include +#include "readInvalidateQuery.h" #include #include #include #include #include +#include "DictionarySourceFactory.h" +#include "DictionaryStructure.h" + namespace DB { @@ -175,4 +178,17 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re } } + +void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) +{ + auto createTableSource = [=](const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + Block & sample_block, + Context & context) -> DictionarySourcePtr { + return std::make_unique(dict_struct, config, config_prefix + ".clickhouse", sample_block, context); + }; + factory.registerSource("clickhouse", createTableSource); +} + } diff --git a/dbms/src/Dictionaries/ClickHouseDictionarySource.h b/dbms/src/Dictionaries/ClickHouseDictionarySource.h index d7559bda99b..89db23737bc 100644 --- a/dbms/src/Dictionaries/ClickHouseDictionarySource.h +++ b/dbms/src/Dictionaries/ClickHouseDictionarySource.h @@ -1,10 +1,9 @@ #pragma once -#include -#include -#include +#include "IDictionarySource.h" +#include "DictionaryStructure.h" +#include "ExternalQueryBuilder.h" #include -#include #include diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp index 86fbfbb474e..61693a3538a 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp @@ -1,5 +1,5 @@ -#include -#include +#include "ComplexKeyCacheDictionary.h" +#include "DictionaryBlockInputStream.h" #include #include #include @@ -9,6 +9,7 @@ #include #include #include +#include "DictionaryFactory.h" namespace ProfileEvents @@ -39,6 +40,7 @@ namespace ErrorCodes extern const int TYPE_MISMATCH; extern const int BAD_ARGUMENTS; extern const int UNSUPPORTED_METHOD; + extern const int TOO_SMALL_BUFFER_SIZE; } @@ -378,4 +380,32 @@ BlockInputStreamPtr ComplexKeyCacheDictionary::getBlockInputStream(const Names & return std::make_shared(shared_from_this(), max_block_size, keys, column_names); } +void registerDictionaryComplexKeyCache(DictionaryFactory & factory) +{ + auto create_layout = [=]( + const std::string & name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr + ) -> DictionaryPtr { + if (!dict_struct.key) + throw Exception {"'key' is required for dictionary of layout 'complex_key_hashed'", ErrorCodes::BAD_ARGUMENTS}; + const auto & layout_prefix = config_prefix + ".layout"; + const auto size = config.getInt(layout_prefix + ".complex_key_cache.size_in_cells"); + if (size == 0) + throw Exception {name + ": dictionary of layout 'cache' cannot have 0 cells", ErrorCodes::TOO_SMALL_BUFFER_SIZE}; + + const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); + if (require_nonempty) + throw Exception {name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set", + ErrorCodes::BAD_ARGUMENTS}; + + const DictionaryLifetime dict_lifetime {config, config_prefix + ".lifetime"}; + return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, size); + }; + factory.registerLayout("complex_key_cache", create_layout); +} + + } diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h index a4a6ae4c16a..f60e142db5e 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h @@ -12,9 +12,9 @@ #include #include #include -#include -#include -#include +#include "DictionaryStructure.h" +#include "IDictionary.h" +#include "IDictionarySource.h" #include #include #include diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp index 3bf10833a80..843c389dcb0 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp @@ -1,4 +1,4 @@ -#include +#include "ComplexKeyCacheDictionary.h" namespace DB { diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp deleted file mode 100644 index c22c14bc267..00000000000 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp +++ /dev/null @@ -1,40 +0,0 @@ -#include "ComplexKeyCacheDictionary.h" - -namespace DB -{ -namespace ErrorCodes -{ - extern const int TYPE_MISMATCH; -} - -#define DECLARE(TYPE) \ - void ComplexKeyCacheDictionary::get##TYPE( \ - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType & out) const \ - { \ - dict_struct.validateKeyTypes(key_types); \ - \ - auto & attribute = getAttribute(attribute_name); \ - if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE)) \ - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \ - ErrorCodes::TYPE_MISMATCH}; \ - \ - const auto null_value = std::get(attribute.null_values); \ - \ - getItemsNumber(attribute, key_columns, out, [&](const size_t) { return null_value; }); \ - } -DECLARE(UInt8) -DECLARE(UInt16) -DECLARE(UInt32) -DECLARE(UInt64) -DECLARE(UInt128) -DECLARE(Int8) -DECLARE(Int16) -DECLARE(Int32) -DECLARE(Int64) -DECLARE(Float32) -DECLARE(Float64) -DECLARE(Decimal32) -DECLARE(Decimal64) -DECLARE(Decimal128) -#undef DECLARE -} diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp.in b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp.in new file mode 100644 index 00000000000..f24c278a554 --- /dev/null +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp.in @@ -0,0 +1,24 @@ +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; +} + +using TYPE = @NAME@; +void ComplexKeyCacheDictionary::get@NAME@(const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType & out) const +{ + dict_struct.validateKeyTypes(key_types); + + auto & attribute = getAttribute(attribute_name); + if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::@NAME@)) + throw Exception {name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), + ErrorCodes::TYPE_MISMATCH}; + + const auto null_value = std::get(attribute.null_values); + + getItemsNumber(attribute, key_columns, out, [&](const size_t) { return null_value; }); +} +} diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp deleted file mode 100644 index 8b7df84288d..00000000000 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp +++ /dev/null @@ -1,41 +0,0 @@ -#include "ComplexKeyCacheDictionary.h" - -namespace DB -{ -namespace ErrorCodes -{ - extern const int TYPE_MISMATCH; -} - -#define DECLARE(TYPE) \ - void ComplexKeyCacheDictionary::get##TYPE(const std::string & attribute_name, \ - const Columns & key_columns, \ - const DataTypes & key_types, \ - const PaddedPODArray & def, \ - ResultArrayType & out) const \ - { \ - dict_struct.validateKeyTypes(key_types); \ - \ - auto & attribute = getAttribute(attribute_name); \ - if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE)) \ - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \ - ErrorCodes::TYPE_MISMATCH}; \ - \ - getItemsNumber(attribute, key_columns, out, [&](const size_t row) { return def[row]; }); \ - } -DECLARE(UInt8) -DECLARE(UInt16) -DECLARE(UInt32) -DECLARE(UInt64) -DECLARE(UInt128) -DECLARE(Int8) -DECLARE(Int16) -DECLARE(Int32) -DECLARE(Int64) -DECLARE(Float32) -DECLARE(Float64) -DECLARE(Decimal32) -DECLARE(Decimal64) -DECLARE(Decimal128) -#undef DECLARE -} diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp.in b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp.in new file mode 100644 index 00000000000..3ec01d96882 --- /dev/null +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp.in @@ -0,0 +1,27 @@ +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; +} + +using TYPE = @NAME@; + +void ComplexKeyCacheDictionary::get@NAME@(const std::string & attribute_name, + const Columns & key_columns, + const DataTypes & key_types, + const PaddedPODArray & def, + ResultArrayType & out) const +{ + dict_struct.validateKeyTypes(key_types); + + auto & attribute = getAttribute(attribute_name); + if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::@NAME@)) + throw Exception {name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), + ErrorCodes::TYPE_MISMATCH}; + + getItemsNumber(attribute, key_columns, out, [&](const size_t row) { return def[row]; }); +} +} diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp deleted file mode 100644 index ecc8794554b..00000000000 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp +++ /dev/null @@ -1,41 +0,0 @@ -#include "ComplexKeyCacheDictionary.h" - -namespace DB -{ -namespace ErrorCodes -{ - extern const int TYPE_MISMATCH; -} - -#define DECLARE(TYPE) \ - void ComplexKeyCacheDictionary::get##TYPE(const std::string & attribute_name, \ - const Columns & key_columns, \ - const DataTypes & key_types, \ - const TYPE def, \ - ResultArrayType & out) const \ - { \ - dict_struct.validateKeyTypes(key_types); \ - \ - auto & attribute = getAttribute(attribute_name); \ - if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE)) \ - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \ - ErrorCodes::TYPE_MISMATCH}; \ - \ - getItemsNumber(attribute, key_columns, out, [&](const size_t) { return def; }); \ - } -DECLARE(UInt8) -DECLARE(UInt16) -DECLARE(UInt32) -DECLARE(UInt64) -DECLARE(UInt128) -DECLARE(Int8) -DECLARE(Int16) -DECLARE(Int32) -DECLARE(Int64) -DECLARE(Float32) -DECLARE(Float64) -DECLARE(Decimal32) -DECLARE(Decimal64) -DECLARE(Decimal128) -#undef DECLARE -} diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp.in b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp.in new file mode 100644 index 00000000000..287fa25c399 --- /dev/null +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp.in @@ -0,0 +1,27 @@ +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; +} + +using TYPE = @NAME@; + +void ComplexKeyCacheDictionary::get@NAME@(const std::string & attribute_name, + const Columns & key_columns, + const DataTypes & key_types, + const TYPE def, + ResultArrayType & out) const +{ + dict_struct.validateKeyTypes(key_types); + + auto & attribute = getAttribute(attribute_name); + if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::@NAME@)) + throw Exception {name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), + ErrorCodes::TYPE_MISMATCH}; + + getItemsNumber(attribute, key_columns, out, [&](const size_t) { return def; }); +} +} diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setAttributeValue.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setAttributeValue.cpp index e85f96de420..9a3d34eb2c7 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setAttributeValue.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setAttributeValue.cpp @@ -1,4 +1,4 @@ -#include +#include "ComplexKeyCacheDictionary.h" namespace DB { diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp index e3af300767d..7477e01da9c 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp @@ -1,4 +1,4 @@ -#include +#include "ComplexKeyCacheDictionary.h" namespace DB { diff --git a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp index 7e52b572c96..cdf01668bd2 100644 --- a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp +++ b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp @@ -1,8 +1,8 @@ #include #include -#include -#include - +#include "ComplexKeyHashedDictionary.h" +#include "DictionaryBlockInputStream.h" +#include "DictionaryFactory.h" namespace DB { @@ -661,5 +661,24 @@ BlockInputStreamPtr ComplexKeyHashedDictionary::getBlockInputStream(const Names return std::make_shared(shared_from_this(), max_block_size, getKeys(), column_names); } +void registerDictionaryComplexKeyHashed(DictionaryFactory & factory) +{ + auto create_layout = [=]( + const std::string & name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr + ) -> DictionaryPtr { + if (!dict_struct.key) + throw Exception {"'key' is required for dictionary of layout 'complex_key_hashed'", ErrorCodes::BAD_ARGUMENTS}; + + const DictionaryLifetime dict_lifetime {config, config_prefix + ".lifetime"}; + const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); + return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + }; + factory.registerLayout("complex_key_hashed", create_layout); +} + } diff --git a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h index 7dd5d5baff9..859266fb5d1 100644 --- a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h @@ -1,8 +1,8 @@ #pragma once -#include -#include -#include +#include "IDictionary.h" +#include "IDictionarySource.h" +#include "DictionaryStructure.h" #include #include #include @@ -243,5 +243,4 @@ private: BlockPtr saved_block; }; - } diff --git a/dbms/src/Dictionaries/DictionaryBlockInputStream.h b/dbms/src/Dictionaries/DictionaryBlockInputStream.h index 8f9a3a10d35..f1778a9fa6d 100644 --- a/dbms/src/Dictionaries/DictionaryBlockInputStream.h +++ b/dbms/src/Dictionaries/DictionaryBlockInputStream.h @@ -6,9 +6,9 @@ #include #include #include -#include -#include -#include +#include "DictionaryBlockInputStreamBase.h" +#include "DictionaryStructure.h" +#include "IDictionary.h" #include #include #include diff --git a/dbms/src/Dictionaries/DictionaryBlockInputStreamBase.cpp b/dbms/src/Dictionaries/DictionaryBlockInputStreamBase.cpp index 4cbb50f32c0..6ef0ca5beac 100644 --- a/dbms/src/Dictionaries/DictionaryBlockInputStreamBase.cpp +++ b/dbms/src/Dictionaries/DictionaryBlockInputStreamBase.cpp @@ -1,4 +1,4 @@ -#include +#include "DictionaryBlockInputStreamBase.h" namespace DB { diff --git a/dbms/src/Dictionaries/DictionaryFactory.cpp b/dbms/src/Dictionaries/DictionaryFactory.cpp new file mode 100644 index 00000000000..e9279de23ec --- /dev/null +++ b/dbms/src/Dictionaries/DictionaryFactory.cpp @@ -0,0 +1,51 @@ +#include "DictionaryFactory.h" + +#include +#include "DictionarySourceFactory.h" +#include "DictionaryStructure.h" + +namespace DB +{ +namespace ErrorCodes +{ + extern const int EXCESSIVE_ELEMENT_IN_CONFIG; + extern const int UNKNOWN_ELEMENT_IN_CONFIG; +} + +void DictionaryFactory::registerLayout(const std::string & layout_type, Creator create_layout) +{ + //LOG_DEBUG(log, "Register dictionary layout type `" + layout_type + "`"); + if (!registered_layouts.emplace(layout_type, std::move(create_layout)).second) + throw Exception("DictionaryFactory: the layout name '" + layout_type + "' is not unique", ErrorCodes::LOGICAL_ERROR); +} + + +DictionaryPtr DictionaryFactory::create( + const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Context & context) const +{ + Poco::Util::AbstractConfiguration::Keys keys; + const auto & layout_prefix = config_prefix + ".layout"; + config.keys(layout_prefix, keys); + if (keys.size() != 1) + throw Exception {name + ": element dictionary.layout should have exactly one child element", + ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG}; + + const DictionaryStructure dict_struct {config, config_prefix + ".structure"}; + + auto source_ptr = DictionarySourceFactory::instance().create(name, config, config_prefix + ".source", dict_struct, context); + + const auto & layout_type = keys.front(); + + { + const auto found = registered_layouts.find(layout_type); + if (found != registered_layouts.end()) + { + const auto & create_layout = found->second; + return create_layout(name, dict_struct, config, config_prefix, std::move(source_ptr)); + } + } + + throw Exception {name + ": unknown dictionary layout type: " + layout_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG}; +} + +} diff --git a/dbms/src/Dictionaries/DictionaryFactory.h b/dbms/src/Dictionaries/DictionaryFactory.h index bd8f6d18af2..2c101425f41 100644 --- a/dbms/src/Dictionaries/DictionaryFactory.h +++ b/dbms/src/Dictionaries/DictionaryFactory.h @@ -1,20 +1,41 @@ #pragma once -#include -#include #include +#include "IDictionary.h" +namespace Poco +{ +namespace Util +{ + class AbstractConfiguration; +} + +class Logger; +} namespace DB { - class Context; class DictionaryFactory : public ext::singleton { public: - DictionaryPtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, Context & context) const; + DictionaryPtr + create(const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Context & context) + const; + + using Creator = std::function; + + void registerLayout(const std::string & layout_type, Creator create_layout); + +private: + using LayoutRegistry = std::unordered_map; + LayoutRegistry registered_layouts; }; } diff --git a/dbms/src/Dictionaries/DictionarySourceFactory.cpp b/dbms/src/Dictionaries/DictionarySourceFactory.cpp index d2deb769839..8441c60cf87 100644 --- a/dbms/src/Dictionaries/DictionarySourceFactory.cpp +++ b/dbms/src/Dictionaries/DictionarySourceFactory.cpp @@ -1,41 +1,16 @@ -#include +#include "DictionarySourceFactory.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include -#include -#include -#include - -#include -#if USE_POCO_MONGODB - #include -#endif -#if USE_POCO_SQLODBC || USE_POCO_DATAODBC - #include -#endif -#if USE_MYSQL - #include -#endif - +#include +#include +#include +#include #include - #include +#include "DictionaryStructure.h" namespace DB { - namespace ErrorCodes { extern const int UNKNOWN_ELEMENT_IN_CONFIG; @@ -46,149 +21,78 @@ namespace ErrorCodes namespace { - -Block createSampleBlock(const DictionaryStructure & dict_struct) -{ - Block block; - - if (dict_struct.id) - block.insert(ColumnWithTypeAndName{ColumnUInt64::create(1, 0), std::make_shared(), dict_struct.id->name}); - - if (dict_struct.key) + Block createSampleBlock(const DictionaryStructure & dict_struct) { - for (const auto & attribute : *dict_struct.key) + Block block; + + if (dict_struct.id) + block.insert(ColumnWithTypeAndName {ColumnUInt64::create(1, 0), std::make_shared(), dict_struct.id->name}); + + if (dict_struct.key) + { + for (const auto & attribute : *dict_struct.key) + { + auto column = attribute.type->createColumn(); + column->insertDefault(); + + block.insert(ColumnWithTypeAndName {std::move(column), attribute.type, attribute.name}); + } + } + + if (dict_struct.range_min) + { + for (const auto & attribute : {dict_struct.range_min, dict_struct.range_max}) + { + const auto & type = std::make_shared(attribute->type); + auto column = type->createColumn(); + column->insertDefault(); + + block.insert(ColumnWithTypeAndName {std::move(column), type, attribute->name}); + } + } + + for (const auto & attribute : dict_struct.attributes) { auto column = attribute.type->createColumn(); - column->insertDefault(); + column->insert(attribute.null_value); - block.insert(ColumnWithTypeAndName{std::move(column), attribute.type, attribute.name}); + block.insert(ColumnWithTypeAndName {std::move(column), attribute.type, attribute.name}); } + + return block; } - if (dict_struct.range_min) - { - for (const auto & attribute : { dict_struct.range_min, dict_struct.range_max }) - { - const auto & type = std::make_shared(attribute->type); - auto column = type->createColumn(); - column->insertDefault(); - - block.insert(ColumnWithTypeAndName{std::move(column), type, attribute->name}); - } - } - - for (const auto & attribute : dict_struct.attributes) - { - auto column = attribute.type->createColumn(); - column->insert(attribute.null_value); - - block.insert(ColumnWithTypeAndName{std::move(column), attribute.type, attribute.name}); - } - - return block; -} - } -DictionarySourceFactory::DictionarySourceFactory() - : log(&Poco::Logger::get("DictionarySourceFactory")) +DictionarySourceFactory::DictionarySourceFactory() : log(&Poco::Logger::get("DictionarySourceFactory")) { -#if USE_POCO_SQLODBC || USE_POCO_DATAODBC - Poco::Data::ODBC::Connector::registerConnector(); -#endif } void DictionarySourceFactory::registerSource(const std::string & source_type, Creator create_source) { LOG_DEBUG(log, "Register dictionary source type `" + source_type + "`"); if (!registered_sources.emplace(source_type, std::move(create_source)).second) - throw Exception("DictionarySourceFactory: the source name '" + source_type + "' is not unique", - ErrorCodes::LOGICAL_ERROR); + throw Exception("DictionarySourceFactory: the source name '" + source_type + "' is not unique", ErrorCodes::LOGICAL_ERROR); } DictionarySourcePtr DictionarySourceFactory::create( - const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - const DictionaryStructure & dict_struct, Context & context) const + const std::string & name, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const DictionaryStructure & dict_struct, + Context & context) const { Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix, keys); if (keys.size() != 1) - throw Exception{name +": element dictionary.source should have exactly one child element", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG}; + throw Exception {name + ": element dictionary.source should have exactly one child element", + ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG}; auto sample_block = createSampleBlock(dict_struct); const auto & source_type = keys.front(); - if ("file" == source_type) - { - if (dict_struct.has_expressions) - throw Exception{"Dictionary source of type `file` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR}; - - const auto filename = config.getString(config_prefix + ".file.path"); - const auto format = config.getString(config_prefix + ".file.format"); - return std::make_unique(filename, format, sample_block, context); - } - else if ("mysql" == source_type) - { -#if USE_MYSQL - return std::make_unique(dict_struct, config, config_prefix + ".mysql", sample_block); -#else - throw Exception{"Dictionary source of type `mysql` is disabled because ClickHouse was built without mysql support.", - ErrorCodes::SUPPORT_IS_DISABLED}; -#endif - } - else if ("clickhouse" == source_type) - { - return std::make_unique(dict_struct, config, config_prefix + ".clickhouse", - sample_block, context); - } - else if ("mongodb" == source_type) - { -#if USE_POCO_MONGODB - return std::make_unique(dict_struct, config, config_prefix + ".mongodb", sample_block); -#else - throw Exception{"Dictionary source of type `mongodb` is disabled because poco library was built without mongodb support.", - ErrorCodes::SUPPORT_IS_DISABLED}; -#endif - } - else if ("odbc" == source_type) - { -#if USE_POCO_SQLODBC || USE_POCO_DATAODBC - BridgeHelperPtr bridge = std::make_shared>(context, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string")); - return std::make_unique(dict_struct, config, config_prefix + ".odbc", sample_block, context, bridge); -#else - throw Exception{"Dictionary source of type `odbc` is disabled because poco library was built without ODBC support.", - ErrorCodes::SUPPORT_IS_DISABLED}; -#endif - } - else if ("jdbc" == source_type) - { - throw Exception{"Dictionary source of type `jdbc` is disabled until consistent support for nullable fields.", - ErrorCodes::SUPPORT_IS_DISABLED}; -// BridgeHelperPtr bridge = std::make_shared>(config, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".connection_string")); -// return std::make_unique(dict_struct, config, config_prefix + ".jdbc", sample_block, context, bridge); - } - else if ("executable" == source_type) - { - if (dict_struct.has_expressions) - throw Exception{"Dictionary source of type `executable` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR}; - - return std::make_unique(dict_struct, config, config_prefix + ".executable", sample_block, context); - } - else if ("http" == source_type) - { - - if (dict_struct.has_expressions) - throw Exception{"Dictionary source of type `http` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR}; - - return std::make_unique(dict_struct, config, config_prefix + ".http", sample_block, context); - } - else if ("library" == source_type) - { - return std::make_unique(dict_struct, config, config_prefix + ".library", sample_block, context); - } - else { const auto found = registered_sources.find(source_type); if (found != registered_sources.end()) @@ -198,7 +102,7 @@ DictionarySourcePtr DictionarySourceFactory::create( } } - throw Exception{name + ": unknown dictionary source type: " + source_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG}; + throw Exception {name + ": unknown dictionary source type: " + source_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG}; } } diff --git a/dbms/src/Dictionaries/DictionarySourceFactory.h b/dbms/src/Dictionaries/DictionarySourceFactory.h index 912a977a2de..1ac6e70f859 100644 --- a/dbms/src/Dictionaries/DictionarySourceFactory.h +++ b/dbms/src/Dictionaries/DictionarySourceFactory.h @@ -1,23 +1,22 @@ #pragma once -#include -#include +#include "IDictionarySource.h" #include +#include namespace Poco { - namespace Util - { - class AbstractConfiguration; - } +namespace Util +{ + class AbstractConfiguration; +} - class Logger; +class Logger; } namespace DB { - class Context; struct DictionaryStructure; @@ -30,15 +29,18 @@ public: const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - const Context & context)>; + Context & context)>; DictionarySourceFactory(); void registerSource(const std::string & source_type, Creator create_source); DictionarySourcePtr create( - const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - const DictionaryStructure & dict_struct, Context & context) const; + const std::string & name, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const DictionaryStructure & dict_struct, + Context & context) const; private: using SourceRegistry = std::unordered_map; diff --git a/dbms/src/Dictionaries/DictionarySourceHelpers.cpp b/dbms/src/Dictionaries/DictionarySourceHelpers.cpp index 73bc6486e7a..108f64cc0bf 100644 --- a/dbms/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/dbms/src/Dictionaries/DictionarySourceHelpers.cpp @@ -1,5 +1,5 @@ -#include -#include +#include "DictionarySourceHelpers.h" +#include "DictionaryStructure.h" #include #include #include diff --git a/dbms/src/Dictionaries/DictionaryStructure.cpp b/dbms/src/Dictionaries/DictionaryStructure.cpp index 4573b90d7ad..c0255c9de7e 100644 --- a/dbms/src/Dictionaries/DictionaryStructure.cpp +++ b/dbms/src/Dictionaries/DictionaryStructure.cpp @@ -1,4 +1,4 @@ -#include +#include "DictionaryStructure.h" #include #include #include diff --git a/dbms/src/Dictionaries/Embedded/CMakeLists.txt b/dbms/src/Dictionaries/Embedded/CMakeLists.txt new file mode 100644 index 00000000000..2af439c9677 --- /dev/null +++ b/dbms/src/Dictionaries/Embedded/CMakeLists.txt @@ -0,0 +1,5 @@ +include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) +add_headers_and_sources(clickhouse_dictionaries_embedded .) +add_headers_and_sources(clickhouse_dictionaries_embedded GeodataProviders) +add_library(clickhouse_dictionaries_embedded ${LINK_MODE} ${clickhouse_dictionaries_embedded_sources}) +target_link_libraries(clickhouse_dictionaries_embedded PRIVATE clickhouse_common_io ${MYSQLXX_LIBRARY}) diff --git a/dbms/src/Dictionaries/Embedded/GeoDictionariesLoader.cpp b/dbms/src/Dictionaries/Embedded/GeoDictionariesLoader.cpp index 2d2967e72a1..0932038ea38 100644 --- a/dbms/src/Dictionaries/Embedded/GeoDictionariesLoader.cpp +++ b/dbms/src/Dictionaries/Embedded/GeoDictionariesLoader.cpp @@ -1,8 +1,8 @@ -#include - -#include -#include +#include "GeoDictionariesLoader.h" +#include +#include "GeodataProviders/HierarchiesProvider.h" +#include "GeodataProviders/NamesProvider.h" std::unique_ptr GeoDictionariesLoader::reloadRegionsHierarchies( const Poco::Util::AbstractConfiguration & config) diff --git a/dbms/src/Dictionaries/Embedded/GeoDictionariesLoader.h b/dbms/src/Dictionaries/Embedded/GeoDictionariesLoader.h index 312637aea90..af7dff31ad3 100644 --- a/dbms/src/Dictionaries/Embedded/GeoDictionariesLoader.h +++ b/dbms/src/Dictionaries/Embedded/GeoDictionariesLoader.h @@ -1,6 +1,6 @@ #pragma once -#include +#include "IGeoDictionariesLoader.h" // Default implementation of geo dictionaries loader used by native server application diff --git a/dbms/src/Dictionaries/Embedded/GeodataProviders/Entries.h b/dbms/src/Dictionaries/Embedded/GeodataProviders/Entries.h index 8e68095fc65..4cfdbd0d75f 100644 --- a/dbms/src/Dictionaries/Embedded/GeodataProviders/Entries.h +++ b/dbms/src/Dictionaries/Embedded/GeodataProviders/Entries.h @@ -1,6 +1,6 @@ #pragma once -#include +#include "Types.h" #include struct RegionEntry diff --git a/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp b/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp index 062c65a16a6..26653f1e905 100644 --- a/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp +++ b/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp @@ -1,8 +1,7 @@ -#include -#include +#include "HierarchiesProvider.h" +#include "HierarchyFormatReader.h" #include - #include #include #include diff --git a/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.h b/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.h index a8f956b1bd3..7025a8e5560 100644 --- a/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.h +++ b/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.h @@ -1,9 +1,8 @@ #pragma once -#include +#include "IHierarchiesProvider.h" #include - #include diff --git a/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.cpp b/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.cpp index 5716879ce92..ace4a1adaa0 100644 --- a/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.cpp +++ b/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.cpp @@ -1,4 +1,4 @@ -#include +#include "HierarchyFormatReader.h" #include #include diff --git a/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.h b/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.h index 1d20c65f62a..e120785a9fa 100644 --- a/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.h +++ b/dbms/src/Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.h @@ -1,7 +1,6 @@ #pragma once -#include - +#include "IHierarchiesProvider.h" #include diff --git a/dbms/src/Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h b/dbms/src/Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h index d12fc312dcf..a8cc915f33f 100644 --- a/dbms/src/Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h +++ b/dbms/src/Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h @@ -1,7 +1,6 @@ #pragma once -#include - +#include "Entries.h" #include #include #include diff --git a/dbms/src/Dictionaries/Embedded/GeodataProviders/INamesProvider.h b/dbms/src/Dictionaries/Embedded/GeodataProviders/INamesProvider.h index fb18684b3fa..bb529a9557b 100644 --- a/dbms/src/Dictionaries/Embedded/GeodataProviders/INamesProvider.h +++ b/dbms/src/Dictionaries/Embedded/GeodataProviders/INamesProvider.h @@ -1,7 +1,6 @@ #pragma once -#include - +#include "Entries.h" #include diff --git a/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.cpp b/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.cpp index df198ebde54..9d0c57f18eb 100644 --- a/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.cpp +++ b/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.cpp @@ -1,4 +1,4 @@ -#include +#include "NamesFormatReader.h" #include diff --git a/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.h b/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.h index d89ccfc3cbe..3f3063be25d 100644 --- a/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.h +++ b/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesFormatReader.h @@ -1,7 +1,6 @@ #pragma once -#include - +#include "INamesProvider.h" #include diff --git a/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp b/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp index ef44c0a4e03..c9042f33788 100644 --- a/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp +++ b/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp @@ -1,6 +1,6 @@ -#include -#include +#include "NamesProvider.h" +#include "NamesFormatReader.h" #include diff --git a/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.h b/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.h index 916dfe38230..937b679c65d 100644 --- a/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.h +++ b/dbms/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.h @@ -1,7 +1,6 @@ #pragma once -#include - +#include "INamesProvider.h" #include diff --git a/dbms/src/Dictionaries/Embedded/IGeoDictionariesLoader.h b/dbms/src/Dictionaries/Embedded/IGeoDictionariesLoader.h index 548ccaf9258..d2637470f96 100644 --- a/dbms/src/Dictionaries/Embedded/IGeoDictionariesLoader.h +++ b/dbms/src/Dictionaries/Embedded/IGeoDictionariesLoader.h @@ -1,12 +1,19 @@ #pragma once -#include -#include - -#include - +#include "RegionsHierarchies.h" +#include "RegionsNames.h" #include +namespace Poco +{ + namespace Util + { + class AbstractConfiguration; + } + + class Logger; +} + // Provides actual versions of geo dictionaries (regions hierarchies, regions names) // Bind data structures (RegionsHierarchies, RegionsNames) with data providers diff --git a/dbms/src/Dictionaries/Embedded/RegionsHierarchies.cpp b/dbms/src/Dictionaries/Embedded/RegionsHierarchies.cpp index d128ea53896..0a1a2928d65 100644 --- a/dbms/src/Dictionaries/Embedded/RegionsHierarchies.cpp +++ b/dbms/src/Dictionaries/Embedded/RegionsHierarchies.cpp @@ -1,7 +1,6 @@ -#include +#include "RegionsHierarchies.h" #include - #include diff --git a/dbms/src/Dictionaries/Embedded/RegionsHierarchies.h b/dbms/src/Dictionaries/Embedded/RegionsHierarchies.h index 8d05fa15d6e..d2b4ee5d48b 100644 --- a/dbms/src/Dictionaries/Embedded/RegionsHierarchies.h +++ b/dbms/src/Dictionaries/Embedded/RegionsHierarchies.h @@ -1,10 +1,8 @@ #pragma once -#include -#include - +#include "RegionsHierarchy.h" +#include "GeodataProviders/IHierarchiesProvider.h" #include - #include diff --git a/dbms/src/Dictionaries/Embedded/RegionsHierarchy.cpp b/dbms/src/Dictionaries/Embedded/RegionsHierarchy.cpp index 2a277e10fe2..3257eb355eb 100644 --- a/dbms/src/Dictionaries/Embedded/RegionsHierarchy.cpp +++ b/dbms/src/Dictionaries/Embedded/RegionsHierarchy.cpp @@ -1,12 +1,10 @@ -#include -#include +#include "RegionsHierarchy.h" +#include "GeodataProviders/IHierarchiesProvider.h" #include #include - #include #include - #include diff --git a/dbms/src/Dictionaries/Embedded/RegionsHierarchy.h b/dbms/src/Dictionaries/Embedded/RegionsHierarchy.h index 838a99f71a1..3e81087629b 100644 --- a/dbms/src/Dictionaries/Embedded/RegionsHierarchy.h +++ b/dbms/src/Dictionaries/Embedded/RegionsHierarchy.h @@ -1,7 +1,6 @@ #pragma once -#include - +#include "GeodataProviders/IHierarchiesProvider.h" #include #include #include diff --git a/dbms/src/Dictionaries/Embedded/RegionsNames.cpp b/dbms/src/Dictionaries/Embedded/RegionsNames.cpp index 2c48f1f18c2..81614610f3a 100644 --- a/dbms/src/Dictionaries/Embedded/RegionsNames.cpp +++ b/dbms/src/Dictionaries/Embedded/RegionsNames.cpp @@ -1,11 +1,9 @@ -#include -#include +#include "RegionsNames.h" +#include "GeodataProviders/INamesProvider.h" #include #include - #include - #include namespace DB diff --git a/dbms/src/Dictionaries/Embedded/RegionsNames.h b/dbms/src/Dictionaries/Embedded/RegionsNames.h index 2cab42d9cd2..543e985fbd7 100644 --- a/dbms/src/Dictionaries/Embedded/RegionsNames.h +++ b/dbms/src/Dictionaries/Embedded/RegionsNames.h @@ -1,12 +1,9 @@ #pragma once -#include - +#include "GeodataProviders/INamesProvider.h" #include - #include #include - #include #include diff --git a/dbms/src/Dictionaries/Embedded/TechDataHierarchy.cpp b/dbms/src/Dictionaries/Embedded/TechDataHierarchy.cpp index 741d3833c34..841d602c22e 100644 --- a/dbms/src/Dictionaries/Embedded/TechDataHierarchy.cpp +++ b/dbms/src/Dictionaries/Embedded/TechDataHierarchy.cpp @@ -1,7 +1,7 @@ #include #if USE_MYSQL -#include +#include "TechDataHierarchy.h" #include #include diff --git a/dbms/src/Dictionaries/Embedded/TechDataHierarchy.h b/dbms/src/Dictionaries/Embedded/TechDataHierarchy.h index 060180bcc44..439ece8f6ca 100644 --- a/dbms/src/Dictionaries/Embedded/TechDataHierarchy.h +++ b/dbms/src/Dictionaries/Embedded/TechDataHierarchy.h @@ -1,12 +1,18 @@ #pragma once -#include -#include - #include - #include +namespace Poco +{ + namespace Util + { + class AbstractConfiguration; + } + + class Logger; +} + /** @brief Class that lets you know if a search engine or operating system belongs * another search engine or operating system, respectively. diff --git a/dbms/src/Dictionaries/ExecutableDictionarySource.cpp b/dbms/src/Dictionaries/ExecutableDictionarySource.cpp index c17ce3e2461..0a04ec1a981 100644 --- a/dbms/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/dbms/src/Dictionaries/ExecutableDictionarySource.cpp @@ -1,12 +1,15 @@ +#include "ExecutableDictionarySource.h" + #include #include -#include #include #include #include -#include +#include "DictionarySourceHelpers.h" #include #include +#include "DictionarySourceFactory.h" +#include "DictionaryStructure.h" namespace DB @@ -229,4 +232,19 @@ std::string ExecutableDictionarySource::toString() const return "Executable: " + command; } +void registerDictionarySourceExecutable(DictionarySourceFactory & factory) +{ + auto createTableSource = [=](const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + Block & sample_block, + const Context & context) -> DictionarySourcePtr { + if (dict_struct.has_expressions) + throw Exception {"Dictionary source of type `executable` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR}; + + return std::make_unique(dict_struct, config, config_prefix + ".executable", sample_block, context); + }; + factory.registerSource("executable", createTableSource); +} + } diff --git a/dbms/src/Dictionaries/ExecutableDictionarySource.h b/dbms/src/Dictionaries/ExecutableDictionarySource.h index a6fe3373aae..00652ad28d9 100644 --- a/dbms/src/Dictionaries/ExecutableDictionarySource.h +++ b/dbms/src/Dictionaries/ExecutableDictionarySource.h @@ -1,7 +1,7 @@ #pragma once -#include -#include +#include "IDictionarySource.h" +#include "DictionaryStructure.h" namespace Poco { class Logger; } diff --git a/dbms/src/Dictionaries/ExternalQueryBuilder.cpp b/dbms/src/Dictionaries/ExternalQueryBuilder.cpp index bb420fdec8f..e8194344848 100644 --- a/dbms/src/Dictionaries/ExternalQueryBuilder.cpp +++ b/dbms/src/Dictionaries/ExternalQueryBuilder.cpp @@ -3,9 +3,9 @@ #include #include #include -#include -#include -#include +#include "writeParenthesisedString.h" +#include "DictionaryStructure.h" +#include "ExternalQueryBuilder.h" namespace DB diff --git a/dbms/src/Dictionaries/ExternalResultDescription.cpp b/dbms/src/Dictionaries/ExternalResultDescription.cpp index a997cb4d07e..2bb903148e1 100644 --- a/dbms/src/Dictionaries/ExternalResultDescription.cpp +++ b/dbms/src/Dictionaries/ExternalResultDescription.cpp @@ -1,5 +1,5 @@ #include -#include +#include "ExternalResultDescription.h" #include #include #include diff --git a/dbms/src/Dictionaries/FileDictionarySource.cpp b/dbms/src/Dictionaries/FileDictionarySource.cpp index 6f37ae2840f..b135ac98c2c 100644 --- a/dbms/src/Dictionaries/FileDictionarySource.cpp +++ b/dbms/src/Dictionaries/FileDictionarySource.cpp @@ -1,9 +1,11 @@ +#include "FileDictionarySource.h" + #include #include -#include #include #include - +#include "DictionarySourceFactory.h" +#include "DictionaryStructure.h" namespace DB { @@ -46,4 +48,23 @@ Poco::Timestamp FileDictionarySource::getLastModification() const return Poco::File{filename}.getLastModified(); } +void registerDictionarySourceFile(DictionarySourceFactory & factory) +{ + auto createTableSource = [=](const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + Block & sample_block, + const Context & context) -> DictionarySourcePtr { + if (dict_struct.has_expressions) + throw Exception {"Dictionary source of type `file` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR}; + + const auto filename = config.getString(config_prefix + ".file.path"); + const auto format = config.getString(config_prefix + ".file.format"); + + return std::make_unique(filename, format, sample_block, context); + }; + + factory.registerSource("file", createTableSource); +} + } diff --git a/dbms/src/Dictionaries/FileDictionarySource.h b/dbms/src/Dictionaries/FileDictionarySource.h index 5dd00223981..fd85eecfb53 100644 --- a/dbms/src/Dictionaries/FileDictionarySource.h +++ b/dbms/src/Dictionaries/FileDictionarySource.h @@ -1,6 +1,6 @@ #pragma once -#include +#include "IDictionarySource.h" #include @@ -9,7 +9,6 @@ namespace DB class Context; - /// Allows loading dictionaries from a file with given format, does not support "random access" class FileDictionarySource final : public IDictionarySource { diff --git a/dbms/src/Dictionaries/FlatDictionary.cpp b/dbms/src/Dictionaries/FlatDictionary.cpp index 47a80ce8bc6..f4fff1008dc 100644 --- a/dbms/src/Dictionaries/FlatDictionary.cpp +++ b/dbms/src/Dictionaries/FlatDictionary.cpp @@ -1,7 +1,7 @@ -#include -#include +#include "FlatDictionary.h" +#include "DictionaryBlockInputStream.h" #include - +#include "DictionaryFactory.h" namespace DB { @@ -14,9 +14,9 @@ namespace ErrorCodes extern const int DICTIONARY_IS_EMPTY; extern const int LOGICAL_ERROR; extern const int UNKNOWN_TYPE; + extern const int UNSUPPORTED_METHOD; } - static const auto initial_array_size = 1024; static const auto max_array_size = 500000; @@ -634,5 +634,33 @@ BlockInputStreamPtr FlatDictionary::getBlockInputStream(const Names & column_nam return std::make_shared(shared_from_this(), max_block_size, getIds() ,column_names); } +void registerDictionaryFlat(DictionaryFactory & factory) +{ + auto create_layout = [=]( + const std::string & name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr + ) -> DictionaryPtr { + + if (dict_struct.key) + throw Exception {"'key' is not supported for dictionary of layout 'flat'", ErrorCodes::UNSUPPORTED_METHOD}; + + if (dict_struct.range_min || dict_struct.range_max) + throw Exception {name + + ": elements .structure.range_min and .structure.range_max should be defined only " + "for a dictionary of layout 'range_hashed'", + ErrorCodes::BAD_ARGUMENTS}; + const DictionaryLifetime dict_lifetime {config, config_prefix + ".lifetime"}; + const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); + return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + + + }; + factory.registerLayout("flat", create_layout); +} + + } diff --git a/dbms/src/Dictionaries/FlatDictionary.h b/dbms/src/Dictionaries/FlatDictionary.h index 272b248d23e..91fb10afa57 100644 --- a/dbms/src/Dictionaries/FlatDictionary.h +++ b/dbms/src/Dictionaries/FlatDictionary.h @@ -1,8 +1,8 @@ #pragma once -#include -#include -#include +#include "IDictionary.h" +#include "IDictionarySource.h" +#include "DictionaryStructure.h" #include #include #include diff --git a/dbms/src/Dictionaries/HTTPDictionarySource.cpp b/dbms/src/Dictionaries/HTTPDictionarySource.cpp index 9099d7de72c..08a6411cc4c 100644 --- a/dbms/src/Dictionaries/HTTPDictionarySource.cpp +++ b/dbms/src/Dictionaries/HTTPDictionarySource.cpp @@ -1,4 +1,4 @@ -#include +#include "HTTPDictionarySource.h" #include #include @@ -6,9 +6,12 @@ #include #include #include -#include +#include "DictionarySourceHelpers.h" #include #include +#include "DictionarySourceFactory.h" +#include "DictionaryStructure.h" + namespace DB { @@ -149,4 +152,19 @@ std::string HTTPDictionarySource::toString() const return uri.toString(); } +void registerDictionarySourceHTTP(DictionarySourceFactory & factory) +{ + auto createTableSource = [=](const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + Block & sample_block, + const Context & context) -> DictionarySourcePtr { + if (dict_struct.has_expressions) + throw Exception {"Dictionary source of type `http` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR}; + + return std::make_unique(dict_struct, config, config_prefix + ".http", sample_block, context); + }; + factory.registerSource("http", createTableSource); +} + } diff --git a/dbms/src/Dictionaries/HTTPDictionarySource.h b/dbms/src/Dictionaries/HTTPDictionarySource.h index ac49cc59e16..531afcc4f6c 100644 --- a/dbms/src/Dictionaries/HTTPDictionarySource.h +++ b/dbms/src/Dictionaries/HTTPDictionarySource.h @@ -1,8 +1,8 @@ #pragma once #include -#include -#include +#include "IDictionarySource.h" +#include "DictionaryStructure.h" #include #include diff --git a/dbms/src/Dictionaries/HashedDictionary.cpp b/dbms/src/Dictionaries/HashedDictionary.cpp index 737bef6b01d..b74f043143d 100644 --- a/dbms/src/Dictionaries/HashedDictionary.cpp +++ b/dbms/src/Dictionaries/HashedDictionary.cpp @@ -1,6 +1,7 @@ #include -#include -#include +#include "HashedDictionary.h" +#include "DictionaryBlockInputStream.h" +#include "DictionaryFactory.h" namespace DB { @@ -11,6 +12,7 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; extern const int BAD_ARGUMENTS; extern const int DICTIONARY_IS_EMPTY; + extern const int UNSUPPORTED_METHOD; } @@ -623,4 +625,29 @@ BlockInputStreamPtr HashedDictionary::getBlockInputStream(const Names & column_n return std::make_shared(shared_from_this(), max_block_size, getIds(), column_names); } +void registerDictionaryHashed(DictionaryFactory & factory) +{ + auto create_layout = [=]( + const std::string & name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr + ) -> DictionaryPtr { + if (dict_struct.key) + throw Exception {"'key' is not supported for dictionary of layout 'hashed'", ErrorCodes::UNSUPPORTED_METHOD}; + + if (dict_struct.range_min || dict_struct.range_max) + throw Exception {name + + ": elements .structure.range_min and .structure.range_max should be defined only " + "for a dictionary of layout 'range_hashed'", + ErrorCodes::BAD_ARGUMENTS}; + const DictionaryLifetime dict_lifetime {config, config_prefix + ".lifetime"}; + const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); + return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + + }; + factory.registerLayout("hashed", create_layout); +} + } diff --git a/dbms/src/Dictionaries/HashedDictionary.h b/dbms/src/Dictionaries/HashedDictionary.h index e54c01e3598..c5269e3f152 100644 --- a/dbms/src/Dictionaries/HashedDictionary.h +++ b/dbms/src/Dictionaries/HashedDictionary.h @@ -1,8 +1,8 @@ #pragma once -#include -#include -#include +#include "IDictionary.h" +#include "IDictionarySource.h" +#include "DictionaryStructure.h" #include #include #include diff --git a/dbms/src/Dictionaries/IDictionary.h b/dbms/src/Dictionaries/IDictionary.h index c2c180979f9..eed6e0b4c71 100644 --- a/dbms/src/Dictionaries/IDictionary.h +++ b/dbms/src/Dictionaries/IDictionary.h @@ -8,7 +8,7 @@ #include #include #include -#include +#include "IDictionarySource.h" namespace DB { diff --git a/dbms/src/Dictionaries/LibraryDictionarySource.cpp b/dbms/src/Dictionaries/LibraryDictionarySource.cpp index a0505ee79f1..f7cbc6e2081 100644 --- a/dbms/src/Dictionaries/LibraryDictionarySource.cpp +++ b/dbms/src/Dictionaries/LibraryDictionarySource.cpp @@ -1,12 +1,15 @@ #include -#include -#include +#include "LibraryDictionarySource.h" +#include "LibraryDictionarySourceExternal.h" #include #include #include #include #include #include +#include "DictionarySourceFactory.h" +#include "DictionaryStructure.h" + namespace DB { @@ -269,4 +272,17 @@ std::string LibraryDictionarySource::toString() const { return path; } + +void registerDictionarySourceLibrary(DictionarySourceFactory & factory) +{ + auto createTableSource = [=](const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + Block & sample_block, + const Context & context) -> DictionarySourcePtr { + return std::make_unique(dict_struct, config, config_prefix + ".library", sample_block, context); + }; + factory.registerSource("library", createTableSource); +} + } diff --git a/dbms/src/Dictionaries/LibraryDictionarySource.h b/dbms/src/Dictionaries/LibraryDictionarySource.h index f59a443784a..4a709d88f5e 100644 --- a/dbms/src/Dictionaries/LibraryDictionarySource.h +++ b/dbms/src/Dictionaries/LibraryDictionarySource.h @@ -1,8 +1,8 @@ #pragma once -#include -#include -#include +#include "DictionaryStructure.h" +#include "ExternalResultDescription.h" +#include "IDictionarySource.h" #include #include @@ -78,4 +78,5 @@ private: std::shared_ptr settings; void * lib_data = nullptr; }; + } diff --git a/dbms/src/Dictionaries/LibraryDictionarySourceExternal.cpp b/dbms/src/Dictionaries/LibraryDictionarySourceExternal.cpp index bcbcb4d0fbc..aa0884d548a 100644 --- a/dbms/src/Dictionaries/LibraryDictionarySourceExternal.cpp +++ b/dbms/src/Dictionaries/LibraryDictionarySourceExternal.cpp @@ -1,4 +1,4 @@ -#include +#include "LibraryDictionarySourceExternal.h" #include namespace diff --git a/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp b/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp index 1f0f91a257f..36a3f0b6055 100644 --- a/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp +++ b/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp @@ -10,8 +10,8 @@ #include #include -#include -#include +#include "DictionaryStructure.h" +#include "MongoDBBlockInputStream.h" #include #include #include diff --git a/dbms/src/Dictionaries/MongoDBBlockInputStream.h b/dbms/src/Dictionaries/MongoDBBlockInputStream.h index 3c964708c91..bcf54107f62 100644 --- a/dbms/src/Dictionaries/MongoDBBlockInputStream.h +++ b/dbms/src/Dictionaries/MongoDBBlockInputStream.h @@ -2,7 +2,7 @@ #include #include -#include +#include "ExternalResultDescription.h" namespace Poco diff --git a/dbms/src/Dictionaries/MongoDBDictionarySource.cpp b/dbms/src/Dictionaries/MongoDBDictionarySource.cpp index e8fe62bd8b6..bbb7893e458 100644 --- a/dbms/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/dbms/src/Dictionaries/MongoDBDictionarySource.cpp @@ -1,21 +1,54 @@ -#include -#if USE_POCO_MONGODB -#include +#include "DictionarySourceFactory.h" +#include "DictionaryStructure.h" +#include "MongoDBDictionarySource.h" +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SUPPORT_IS_DISABLED; +} + +void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) +{ + auto createTableSource = [=](const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + Block & sample_block, + const Context & /* context */) -> DictionarySourcePtr { +#if USE_POCO_MONGODB + return std::make_unique(dict_struct, config, config_prefix + ".mongodb", sample_block); +#else + (void)dict_struct; + (void)config; + (void)config_prefix; + (void)sample_block; + throw Exception {"Dictionary source of type `mongodb` is disabled because poco library was built without mongodb support.", + ErrorCodes::SUPPORT_IS_DISABLED}; +#endif + }; + factory.registerSource("mongodb", createTableSource); +} + +} + + +#if USE_POCO_MONGODB + +#include #include #include #include #include #include - #include // only after poco // naming conflict: // Poco/MongoDB/BSONWriter.h:54: void writeCString(const std::string & value); // dbms/src/IO/WriteHelpers.h:146 #define writeCString(s, buf) -#include -#include +#include "MongoDBBlockInputStream.h" #include #include #include diff --git a/dbms/src/Dictionaries/MongoDBDictionarySource.h b/dbms/src/Dictionaries/MongoDBDictionarySource.h index 92428f818b2..f7c95487a1a 100644 --- a/dbms/src/Dictionaries/MongoDBDictionarySource.h +++ b/dbms/src/Dictionaries/MongoDBDictionarySource.h @@ -1,8 +1,10 @@ #pragma once -#include -#include +#include +#if USE_POCO_MONGODB +#include "IDictionarySource.h" +#include "DictionaryStructure.h" namespace Poco { @@ -79,3 +81,10 @@ private: }; } +#endif + +/*namespace DB +{ +class DictionarySourceFactory; +void registerDictionarySourceMongoDB(DictionarySourceFactory & factory); +}*/ diff --git a/dbms/src/Dictionaries/MySQLBlockInputStream.cpp b/dbms/src/Dictionaries/MySQLBlockInputStream.cpp index 41318e78b83..8016eedb56c 100644 --- a/dbms/src/Dictionaries/MySQLBlockInputStream.cpp +++ b/dbms/src/Dictionaries/MySQLBlockInputStream.cpp @@ -1,7 +1,7 @@ #include #if USE_MYSQL -#include +#include "MySQLBlockInputStream.h" #include #include #include diff --git a/dbms/src/Dictionaries/MySQLBlockInputStream.h b/dbms/src/Dictionaries/MySQLBlockInputStream.h index 7e082fdc21d..9e92f0c03a4 100644 --- a/dbms/src/Dictionaries/MySQLBlockInputStream.h +++ b/dbms/src/Dictionaries/MySQLBlockInputStream.h @@ -2,7 +2,7 @@ #include #include -#include +#include "ExternalResultDescription.h" #include #include #include diff --git a/dbms/src/Dictionaries/MySQLDictionarySource.cpp b/dbms/src/Dictionaries/MySQLDictionarySource.cpp index 63639ceb64c..3fa8804845c 100644 --- a/dbms/src/Dictionaries/MySQLDictionarySource.cpp +++ b/dbms/src/Dictionaries/MySQLDictionarySource.cpp @@ -1,18 +1,51 @@ -#include -#if USE_MYSQL +#include "MySQLDictionarySource.h" +#include "DictionarySourceFactory.h" +#include "DictionaryStructure.h" +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SUPPORT_IS_DISABLED; +} + +void registerDictionarySourceMysql(DictionarySourceFactory & factory) +{ + auto createTableSource = [=](const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + Block & sample_block, + const Context & /* context */) -> DictionarySourcePtr { +#if USE_MYSQL + return std::make_unique(dict_struct, config, config_prefix + ".mysql", sample_block); +#else + (void)dict_struct; + (void)config; + (void)config_prefix; + (void)sample_block; + throw Exception {"Dictionary source of type `mysql` is disabled because ClickHouse was built without mysql support.", + ErrorCodes::SUPPORT_IS_DISABLED}; +#endif + }; + factory.registerSource("mysql", createTableSource); +} + +} + + +#if USE_MYSQL #include #include #include -#include - #include #include - -#include -#include -#include - +#include "MySQLBlockInputStream.h" +#include "readInvalidateQuery.h" #include diff --git a/dbms/src/Dictionaries/MySQLDictionarySource.h b/dbms/src/Dictionaries/MySQLDictionarySource.h index 2b9a63ac194..cda7f72b29c 100644 --- a/dbms/src/Dictionaries/MySQLDictionarySource.h +++ b/dbms/src/Dictionaries/MySQLDictionarySource.h @@ -1,8 +1,11 @@ #pragma once -#include -#include -#include +#include +#if USE_MYSQL + +#include "IDictionarySource.h" +#include "ExternalQueryBuilder.h" +#include "DictionaryStructure.h" #include #include @@ -81,3 +84,5 @@ private: }; } + +#endif diff --git a/dbms/src/Dictionaries/ODBCBlockInputStream.cpp b/dbms/src/Dictionaries/ODBCBlockInputStream.cpp index 85f727963d6..879ca2fa7c2 100644 --- a/dbms/src/Dictionaries/ODBCBlockInputStream.cpp +++ b/dbms/src/Dictionaries/ODBCBlockInputStream.cpp @@ -1,12 +1,9 @@ -#include - +#include "ODBCBlockInputStream.h" #include #include #include - #include #include - #include #include #include diff --git a/dbms/src/Dictionaries/ODBCBlockInputStream.h b/dbms/src/Dictionaries/ODBCBlockInputStream.h index b9f5543c271..a5f6ab872ce 100644 --- a/dbms/src/Dictionaries/ODBCBlockInputStream.h +++ b/dbms/src/Dictionaries/ODBCBlockInputStream.h @@ -2,12 +2,10 @@ #include #include -#include - +#include "ExternalResultDescription.h" #include #include #include - #include diff --git a/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h b/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h index a6c31cf7a9a..a08cc5ca578 100644 --- a/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h +++ b/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h @@ -5,10 +5,10 @@ #include #include #include -#include -#include -#include -#include +#include "DictionaryBlockInputStreamBase.h" +#include "DictionaryStructure.h" +#include "IDictionary.h" +#include "RangeHashedDictionary.h" #include namespace DB diff --git a/dbms/src/Dictionaries/RangeHashedDictionary.cpp b/dbms/src/Dictionaries/RangeHashedDictionary.cpp index 9aeea8eff7c..5db404b793d 100644 --- a/dbms/src/Dictionaries/RangeHashedDictionary.cpp +++ b/dbms/src/Dictionaries/RangeHashedDictionary.cpp @@ -1,10 +1,10 @@ -#include -#include +#include "RangeHashedDictionary.h" +#include "RangeDictionaryBlockInputStream.h" #include #include #include #include - +#include "DictionaryFactory.h" namespace { @@ -50,6 +50,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int DICTIONARY_IS_EMPTY; extern const int TYPE_MISMATCH; + extern const int UNSUPPORTED_METHOD; } bool RangeHashedDictionary::Range::isCorrectDate(const RangeStorageType & date) @@ -559,4 +560,28 @@ BlockInputStreamPtr RangeHashedDictionary::getBlockInputStream(const Names & col return callable.stream; } + +void registerDictionaryRangeHashed(DictionaryFactory & factory) +{ + auto create_layout = [=]( + const std::string & name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr + ) -> DictionaryPtr { + if (dict_struct.key) + throw Exception {"'key' is not supported for dictionary of layout 'range_hashed'", ErrorCodes::UNSUPPORTED_METHOD}; + + if (!dict_struct.range_min || !dict_struct.range_max) + throw Exception {name + ": dictionary of layout 'range_hashed' requires .structure.range_min and .structure.range_max", + ErrorCodes::BAD_ARGUMENTS}; + + const DictionaryLifetime dict_lifetime {config, config_prefix + ".lifetime"}; + const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); + return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + }; + factory.registerLayout("range_hashed", create_layout); +} + } diff --git a/dbms/src/Dictionaries/RangeHashedDictionary.h b/dbms/src/Dictionaries/RangeHashedDictionary.h index 5e8adc4ceae..2b4591b564d 100644 --- a/dbms/src/Dictionaries/RangeHashedDictionary.h +++ b/dbms/src/Dictionaries/RangeHashedDictionary.h @@ -1,8 +1,8 @@ #pragma once -#include -#include -#include +#include "IDictionary.h" +#include "IDictionarySource.h" +#include "DictionaryStructure.h" #include #include #include diff --git a/dbms/src/Dictionaries/TrieDictionary.cpp b/dbms/src/Dictionaries/TrieDictionary.cpp index 615a8df2ae0..047b5932818 100644 --- a/dbms/src/Dictionaries/TrieDictionary.cpp +++ b/dbms/src/Dictionaries/TrieDictionary.cpp @@ -3,16 +3,17 @@ #include #include #include -#include +#include "TrieDictionary.h" #include #include -#include +#include "DictionaryBlockInputStream.h" #include #include #include #include #include #include +#include "DictionaryFactory.h" namespace DB @@ -665,4 +666,25 @@ BlockInputStreamPtr TrieDictionary::getBlockInputStream(const Names & column_nam std::move(getKeys), std::move(getView)); } + +void registerDictionaryTrie(DictionaryFactory & factory) +{ + auto create_layout = [=]( + const std::string & name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr + ) -> DictionaryPtr { + if (!dict_struct.key) + throw Exception {"'key' is required for dictionary of layout 'ip_trie'", ErrorCodes::BAD_ARGUMENTS}; + + const DictionaryLifetime dict_lifetime {config, config_prefix + ".lifetime"}; + const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); + // This is specialised trie for storing IPv4 and IPv6 prefixes. + return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + }; + factory.registerLayout("ip_trie", create_layout); +} + } diff --git a/dbms/src/Dictionaries/TrieDictionary.h b/dbms/src/Dictionaries/TrieDictionary.h index af4879b5a04..90a68983abc 100644 --- a/dbms/src/Dictionaries/TrieDictionary.h +++ b/dbms/src/Dictionaries/TrieDictionary.h @@ -1,8 +1,8 @@ #pragma once -#include -#include -#include +#include "IDictionary.h" +#include "IDictionarySource.h" +#include "DictionaryStructure.h" #include #include #include @@ -242,5 +242,4 @@ private: Logger * logger; }; - } diff --git a/dbms/src/Dictionaries/XDBCDictionarySource.cpp b/dbms/src/Dictionaries/XDBCDictionarySource.cpp index e7285f17025..16833420dfe 100644 --- a/dbms/src/Dictionaries/XDBCDictionarySource.cpp +++ b/dbms/src/Dictionaries/XDBCDictionarySource.cpp @@ -1,4 +1,5 @@ -#include +#include "XDBCDictionarySource.h" + #include #include #include @@ -7,17 +8,28 @@ #include #include #include -#include +#include "readInvalidateQuery.h" #include #include #include #include #include +#include "DictionarySourceFactory.h" +#include "DictionaryStructure.h" +#include +#if USE_POCO_SQLODBC || USE_POCO_DATAODBC + #include +#endif namespace DB { +namespace ErrorCodes +{ + extern const int SUPPORT_IS_DISABLED; +} + namespace { class XDBCBridgeBlockInputStream : public IProfilingBlockInputStream @@ -219,4 +231,42 @@ BlockInputStreamPtr XDBCDictionarySource::loadBase(const std::string & query) co timeouts, bridge_helper->getName() + "BlockInputStream"); } +void registerDictionarySourceXDBC(DictionarySourceFactory & factory) +{ +#if USE_POCO_SQLODBC || USE_POCO_DATAODBC + Poco::Data::ODBC::Connector::registerConnector(); +#endif + + auto createTableSource = [=](const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + Block & sample_block, + Context & context) -> DictionarySourcePtr { +#if USE_POCO_SQLODBC || USE_POCO_DATAODBC + BridgeHelperPtr bridge = std::make_shared>(context, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string")); + return std::make_unique(dict_struct, config, config_prefix + ".odbc", sample_block, context, bridge); +#else + throw Exception {"Dictionary source of type `odbc` is disabled because poco library was built without ODBC support.", + ErrorCodes::SUPPORT_IS_DISABLED}; +#endif + }; + factory.registerSource("odbc", createTableSource); +} + +void registerDictionarySourceJDBC(DictionarySourceFactory & factory) +{ + auto createTableSource = [=](const DictionaryStructure & /* dict_struct */, + const Poco::Util::AbstractConfiguration & /* config */, + const std::string & /* config_prefix */, + Block & /* sample_block */, + const Context & /* context */) -> DictionarySourcePtr { + throw Exception {"Dictionary source of type `jdbc` is disabled until consistent support for nullable fields.", + ErrorCodes::SUPPORT_IS_DISABLED}; + // BridgeHelperPtr bridge = std::make_shared>(config, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".connection_string")); + // return std::make_unique(dict_struct, config, config_prefix + ".jdbc", sample_block, context, bridge); + }; + factory.registerSource("jdbc", createTableSource); +} + + } diff --git a/dbms/src/Dictionaries/XDBCDictionarySource.h b/dbms/src/Dictionaries/XDBCDictionarySource.h index 352b7eecbd3..cee862e8fc3 100644 --- a/dbms/src/Dictionaries/XDBCDictionarySource.h +++ b/dbms/src/Dictionaries/XDBCDictionarySource.h @@ -2,11 +2,9 @@ #include #include - -#include -#include -#include - +#include "DictionaryStructure.h" +#include "ExternalQueryBuilder.h" +#include "IDictionarySource.h" #include #include @@ -83,4 +81,5 @@ private: ConnectionTimeouts timeouts; const Context & global_context; }; + } diff --git a/dbms/src/Dictionaries/readInvalidateQuery.cpp b/dbms/src/Dictionaries/readInvalidateQuery.cpp index 2095f696ea4..cacd7636b9f 100644 --- a/dbms/src/Dictionaries/readInvalidateQuery.cpp +++ b/dbms/src/Dictionaries/readInvalidateQuery.cpp @@ -1,4 +1,4 @@ -#include +#include "readInvalidateQuery.h" #include namespace DB diff --git a/dbms/src/Dictionaries/registerDictionaries.cpp b/dbms/src/Dictionaries/registerDictionaries.cpp new file mode 100644 index 00000000000..1a8c5a7be7b --- /dev/null +++ b/dbms/src/Dictionaries/registerDictionaries.cpp @@ -0,0 +1,52 @@ +#include "DictionaryFactory.h" +#include "DictionarySourceFactory.h" + +namespace DB +{ +void registerDictionarySourceFile(DictionarySourceFactory & source_factory); +void registerDictionarySourceMysql(DictionarySourceFactory & source_factory); +void registerDictionarySourceClickHouse(DictionarySourceFactory & source_factory); +void registerDictionarySourceMongoDB(DictionarySourceFactory & source_factory); +void registerDictionarySourceXDBC(DictionarySourceFactory & source_factory); +void registerDictionarySourceJDBC(DictionarySourceFactory & source_factory); +void registerDictionarySourceExecutable(DictionarySourceFactory & source_factory); +void registerDictionarySourceHTTP(DictionarySourceFactory & source_factory); +void registerDictionarySourceLibrary(DictionarySourceFactory & source_factory); + +void registerDictionaryRangeHashed(DictionaryFactory & factory); +void registerDictionaryComplexKeyHashed(DictionaryFactory & factory); +void registerDictionaryComplexKeyCache(DictionaryFactory & factory); +void registerDictionaryTrie(DictionaryFactory & factory); +void registerDictionaryFlat(DictionaryFactory & factory); +void registerDictionaryHashed(DictionaryFactory & factory); +void registerDictionaryCache(DictionaryFactory & factory); + + +void registerDictionaries() +{ + { + auto & source_factory = DictionarySourceFactory::instance(); + registerDictionarySourceFile(source_factory); + registerDictionarySourceMysql(source_factory); + registerDictionarySourceClickHouse(source_factory); + registerDictionarySourceMongoDB(source_factory); + registerDictionarySourceXDBC(source_factory); + registerDictionarySourceJDBC(source_factory); + registerDictionarySourceExecutable(source_factory); + registerDictionarySourceHTTP(source_factory); + registerDictionarySourceLibrary(source_factory); + } + + { + auto & factory = DictionaryFactory::instance(); + registerDictionaryRangeHashed(factory); + registerDictionaryComplexKeyHashed(factory); + registerDictionaryComplexKeyCache(factory); + registerDictionaryTrie(factory); + registerDictionaryFlat(factory); + registerDictionaryHashed(factory); + registerDictionaryCache(factory); + } +} + +} diff --git a/dbms/src/Dictionaries/registerDictionaries.h b/dbms/src/Dictionaries/registerDictionaries.h new file mode 100644 index 00000000000..e8480277c2c --- /dev/null +++ b/dbms/src/Dictionaries/registerDictionaries.h @@ -0,0 +1,6 @@ +#pragma once + +namespace DB +{ +void registerDictionaries(); +} diff --git a/dbms/src/Dictionaries/writeParenthesisedString.cpp b/dbms/src/Dictionaries/writeParenthesisedString.cpp index e2015d8e604..e162a7820f1 100644 --- a/dbms/src/Dictionaries/writeParenthesisedString.cpp +++ b/dbms/src/Dictionaries/writeParenthesisedString.cpp @@ -1,4 +1,4 @@ -#include +#include "writeParenthesisedString.h" namespace DB { diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index b9567bed775..1307c47260e 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -10,7 +10,18 @@ list(REMOVE_ITEM clickhouse_functions_headers IFunction.h FunctionFactory.h Func add_library(clickhouse_functions ${LINK_MODE} ${clickhouse_functions_sources}) -target_link_libraries(clickhouse_functions PUBLIC dbms PRIVATE ${CONSISTENT_HASHING_LIBRARY} consistent-hashing-sumbur ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES} murmurhash ${BASE64_LIBRARY} ${OPENSSL_CRYPTO_LIBRARY}) +target_link_libraries(clickhouse_functions + PUBLIC + dbms + PRIVATE + clickhouse_dictionaries + ${CONSISTENT_HASHING_LIBRARY} + consistent-hashing-sumbur + ${FARMHASH_LIBRARIES} + ${METROHASH_LIBRARIES} + murmurhash + ${BASE64_LIBRARY} + ${OPENSSL_CRYPTO_LIBRARY}) target_include_directories (clickhouse_functions SYSTEM BEFORE PUBLIC ${DIVIDE_INCLUDE_DIR}) diff --git a/dbms/src/IO/HTTPCommon.cpp b/dbms/src/IO/HTTPCommon.cpp index c6b7e47225c..8e6d52738ea 100644 --- a/dbms/src/IO/HTTPCommon.cpp +++ b/dbms/src/IO/HTTPCommon.cpp @@ -84,6 +84,8 @@ namespace /// doesn't work properly without patch #if POCO_CLICKHOUSE_PATCH session->setKeepAlive(keep_alive); +#else + (void)keep_alive; // Avoid warning: unused parameter #endif return session; diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Interpreters/CatBoostModel.cpp similarity index 99% rename from dbms/src/Dictionaries/CatBoostModel.cpp rename to dbms/src/Interpreters/CatBoostModel.cpp index fd1ed454e0a..61fc1d19785 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Interpreters/CatBoostModel.cpp @@ -1,4 +1,5 @@ -#include +#include "CatBoostModel.h" + #include #include #include diff --git a/dbms/src/Dictionaries/CatBoostModel.h b/dbms/src/Interpreters/CatBoostModel.h similarity index 100% rename from dbms/src/Dictionaries/CatBoostModel.h rename to dbms/src/Interpreters/CatBoostModel.h diff --git a/dbms/src/Interpreters/DictionaryFactory.cpp b/dbms/src/Interpreters/DictionaryFactory.cpp deleted file mode 100644 index 3e3fa774465..00000000000 --- a/dbms/src/Interpreters/DictionaryFactory.cpp +++ /dev/null @@ -1,133 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int EXCESSIVE_ELEMENT_IN_CONFIG; - extern const int UNKNOWN_ELEMENT_IN_CONFIG; - extern const int UNSUPPORTED_METHOD; - extern const int TOO_SMALL_BUFFER_SIZE; - extern const int BAD_ARGUMENTS; -} - - -DictionaryPtr DictionaryFactory::create(const std::string & name, const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, Context & context) const -{ - Poco::Util::AbstractConfiguration::Keys keys; - const auto & layout_prefix = config_prefix + ".layout"; - config.keys(layout_prefix, keys); - if (keys.size() != 1) - throw Exception{name + ": element dictionary.layout should have exactly one child element", - ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG}; - - const DictionaryStructure dict_struct{config, config_prefix + ".structure"}; - - auto source_ptr = DictionarySourceFactory::instance().create( - name, config, config_prefix + ".source", dict_struct, context); - - const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; - - const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); - - const auto & layout_type = keys.front(); - - if ("range_hashed" == layout_type) - { - if (dict_struct.key) - throw Exception{"'key' is not supported for dictionary of layout 'range_hashed'", - ErrorCodes::UNSUPPORTED_METHOD}; - - if (!dict_struct.range_min || !dict_struct.range_max) - throw Exception{name + ": dictionary of layout 'range_hashed' requires .structure.range_min and .structure.range_max", - ErrorCodes::BAD_ARGUMENTS}; - - return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); - } - else if ("complex_key_hashed" == layout_type) - { - if (!dict_struct.key) - throw Exception{"'key' is required for dictionary of layout 'complex_key_hashed'", - ErrorCodes::BAD_ARGUMENTS}; - - return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); - } - else if ("complex_key_cache" == layout_type) - { - if (!dict_struct.key) - throw Exception{"'key' is required for dictionary of layout 'complex_key_hashed'", - ErrorCodes::BAD_ARGUMENTS}; - - const auto size = config.getInt(layout_prefix + ".complex_key_cache.size_in_cells"); - if (size == 0) - throw Exception{name + ": dictionary of layout 'cache' cannot have 0 cells", - ErrorCodes::TOO_SMALL_BUFFER_SIZE}; - - if (require_nonempty) - throw Exception{name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set", - ErrorCodes::BAD_ARGUMENTS}; - - return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, size); - } - else if ("ip_trie" == layout_type) - { - if (!dict_struct.key) - throw Exception{"'key' is required for dictionary of layout 'ip_trie'", - ErrorCodes::BAD_ARGUMENTS}; - - // This is specialised trie for storing IPv4 and IPv6 prefixes. - return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); - } - else - { - if (dict_struct.key) - throw Exception{"'key' is not supported for dictionary of layout '" + layout_type + "'", - ErrorCodes::UNSUPPORTED_METHOD}; - - if (dict_struct.range_min || dict_struct.range_max) - throw Exception{name + ": elements .structure.range_min and .structure.range_max should be defined only " - "for a dictionary of layout 'range_hashed'", - ErrorCodes::BAD_ARGUMENTS}; - - if ("flat" == layout_type) - { - return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); - } - else if ("hashed" == layout_type) - { - return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); - } - else if ("cache" == layout_type) - { - const auto size = config.getInt(layout_prefix + ".cache.size_in_cells"); - if (size == 0) - throw Exception{name + ": dictionary of layout 'cache' cannot have 0 cells", - ErrorCodes::TOO_SMALL_BUFFER_SIZE}; - - if (require_nonempty) - throw Exception{name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set", - ErrorCodes::BAD_ARGUMENTS}; - - return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, size); - } - } - - throw Exception{name + ": unknown dictionary layout type: " + layout_type, - ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG}; -} - - -} diff --git a/dbms/src/Interpreters/ExternalModels.h b/dbms/src/Interpreters/ExternalModels.h index cd0fe102bb1..9a048032bd7 100644 --- a/dbms/src/Interpreters/ExternalModels.h +++ b/dbms/src/Interpreters/ExternalModels.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include diff --git a/dbms/src/Storages/System/StorageSystemModels.cpp b/dbms/src/Storages/System/StorageSystemModels.cpp index 2479742c8ec..0b609eb8b7d 100644 --- a/dbms/src/Storages/System/StorageSystemModels.cpp +++ b/dbms/src/Storages/System/StorageSystemModels.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB { From 05ac77dbdee4d3ef084bbd0e4a3a974763da907a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 28 Nov 2018 14:37:07 +0300 Subject: [PATCH 21/47] Added test with Date and numbers comparsion. --- dbms/src/Functions/FunctionsComparison.h | 2 +- .../0_stateless/00762_date_comparsion.reference | 6 ++++++ .../queries/0_stateless/00762_date_comparsion.sql | 13 +++++++++++++ 3 files changed, 20 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/00762_date_comparsion.reference create mode 100644 dbms/tests/queries/0_stateless/00762_date_comparsion.sql diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index f2b377490da..b6161ad2da7 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -1093,7 +1093,7 @@ public: const DataTypeTuple * right_tuple = checkAndGetDataType(arguments[1].get()); bool both_represented_by_number = arguments[0]->isValueRepresentedByNumber() && arguments[1]->isValueRepresentedByNumber(); - bool has_date = left.isDate() && right.isDate(); + bool has_date = left.isDate() || right.isDate(); if (!((both_represented_by_number && !has_date) /// Do not allow compare date and number. || (left.isStringOrFixedString() && right.isStringOrFixedString()) diff --git a/dbms/tests/queries/0_stateless/00762_date_comparsion.reference b/dbms/tests/queries/0_stateless/00762_date_comparsion.reference new file mode 100644 index 00000000000..79c1756a2cc --- /dev/null +++ b/dbms/tests/queries/0_stateless/00762_date_comparsion.reference @@ -0,0 +1,6 @@ +0 +1 +0 +0 +1 +0 diff --git a/dbms/tests/queries/0_stateless/00762_date_comparsion.sql b/dbms/tests/queries/0_stateless/00762_date_comparsion.sql new file mode 100644 index 00000000000..522d3129720 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00762_date_comparsion.sql @@ -0,0 +1,13 @@ +SET send_logs_level = 'none'; + +select toDate('2018-01-01') < '2018-01-01'; +select toDate('2018-01-01') == '2018-01-01'; +select toDate('2018-01-01') != '2018-01-01'; +select toDate('2018-01-01') < toDate('2018-01-01'); +select toDate('2018-01-01') == toDate('2018-01-01'); +select toDate('2018-01-01') != toDate('2018-01-01'); + +select toDate('2018-01-01') < 1; -- { serverError 43 } +select toDate('2018-01-01') == 1; -- { serverError 43 } +select toDate('2018-01-01') != 1; -- { serverError 43 } + From 107cf2c9b2bcfb6d18a997136843a6db20752b19 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 28 Nov 2018 15:19:01 +0300 Subject: [PATCH 22/47] Add secure option to clickhouse-benchmark and clickhouse-performance-test --- dbms/programs/benchmark/Benchmark.cpp | 6 ++++-- dbms/programs/performance-test/PerformanceTest.cpp | 9 +++++++-- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 8931774bac1..01b205e4f0f 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -50,13 +50,13 @@ class Benchmark { public: Benchmark(unsigned concurrency_, double delay_, - const String & host_, UInt16 port_, const String & default_database_, + const String & host_, UInt16 port_, bool secure_, const String & default_database_, const String & user_, const String & password_, const String & stage, bool randomize_, size_t max_iterations_, double max_time_, const String & json_path_, const ConnectionTimeouts & timeouts, const Settings & settings_) : concurrency(concurrency_), delay(delay_), queue(concurrency), - connections(concurrency, host_, port_, default_database_, user_, password_, timeouts), + connections(concurrency, host_, port_, default_database_, user_, password_, timeouts, "benchmark", Protocol::Compression::Enable, secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable), randomize(randomize_), max_iterations(max_iterations_), max_time(max_time_), json_path(json_path_), settings(settings_), global_context(Context::createGlobal()), pool(concurrency) { @@ -432,6 +432,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ("json", value()->default_value(""), "write final report to specified file in JSON format") ("host,h", value()->default_value("localhost"), "") ("port", value()->default_value(9000), "") + ("secure", value()->default_value(false), "use secure connection") ("user", value()->default_value("default"), "") ("password", value()->default_value(""), "") ("database", value()->default_value("default"), "") @@ -470,6 +471,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) options["delay"].as(), options["host"].as(), options["port"].as(), + options["secure"].as(), options["database"].as(), options["user"].as(), options["password"].as(), diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index ebebedd8769..6e6cedbef9e 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -494,6 +494,7 @@ public: PerformanceTest(const String & host_, const UInt16 port_, + const bool & secure_, const String & default_database_, const String & user_, const String & password_, @@ -507,7 +508,7 @@ public: Strings && tests_names_regexp_, Strings && skip_names_regexp_, const ConnectionTimeouts & timeouts) - : connection(host_, port_, default_database_, user_, password_, timeouts), + : connection(host_, port_, default_database_, user_, password_, timeouts, "performance-test", Protocol::Compression::Enable, secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable), gotSIGINT(false), lite_output(lite_output_), profiles_file(profiles_file_), @@ -1385,7 +1386,10 @@ try boost::program_options::options_description desc("Allowed options"); desc.add_options()("help", "produce help message")("lite", "use lite version of output")( "profiles-file", value()->default_value(""), "Specify a file with global profiles")( - "host,h", value()->default_value("localhost"), "")("port", value()->default_value(9000), "")( + "host,h", value()->default_value("localhost"), "") + ("port", value()->default_value(9000), "") + ("secure", value()->default_value(false), "use secure connection") + ( "database", value()->default_value("default"), "")("user", value()->default_value("default"), "")( "password", value()->default_value(""), "")("tags", value()->multitoken(), "Run only tests with tag")( "skip-tags", value()->multitoken(), "Do not run tests with tag")("names", @@ -1476,6 +1480,7 @@ try DB::PerformanceTest performanceTest(options["host"].as(), options["port"].as(), + options["secure"].as(), options["database"].as(), options["user"].as(), options["password"].as(), From 3fabaa28ebd091b25e8bd0e148736ab8fe8bd423 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 28 Nov 2018 15:20:16 +0300 Subject: [PATCH 23/47] Updated test with Date and numbers comparsion. --- .../tests/queries/0_stateless/00762_date_comparsion.reference | 1 + dbms/tests/queries/0_stateless/00762_date_comparsion.sql | 4 ++++ 2 files changed, 5 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00762_date_comparsion.reference b/dbms/tests/queries/0_stateless/00762_date_comparsion.reference index 79c1756a2cc..e499c07edeb 100644 --- a/dbms/tests/queries/0_stateless/00762_date_comparsion.reference +++ b/dbms/tests/queries/0_stateless/00762_date_comparsion.reference @@ -1,3 +1,4 @@ +1 0 1 0 diff --git a/dbms/tests/queries/0_stateless/00762_date_comparsion.sql b/dbms/tests/queries/0_stateless/00762_date_comparsion.sql index 522d3129720..b874cb0b0b3 100644 --- a/dbms/tests/queries/0_stateless/00762_date_comparsion.sql +++ b/dbms/tests/queries/0_stateless/00762_date_comparsion.sql @@ -1,5 +1,8 @@ SET send_logs_level = 'none'; +select today() < 2018-11-14; -- { serverError 43 } +select toDate('2018-01-01') < '2018-11-14'; + select toDate('2018-01-01') < '2018-01-01'; select toDate('2018-01-01') == '2018-01-01'; select toDate('2018-01-01') != '2018-01-01'; @@ -11,3 +14,4 @@ select toDate('2018-01-01') < 1; -- { serverError 43 } select toDate('2018-01-01') == 1; -- { serverError 43 } select toDate('2018-01-01') != 1; -- { serverError 43 } + From 3c551b43b8f60b1548d172df75636c6c7f1bb875 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Nov 2018 16:40:24 +0300 Subject: [PATCH 24/47] Updated comment [#CLICKHOUSE-2] --- dbms/src/Databases/IDatabase.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index 1bf2aab99ac..7b7b877b0e1 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -91,7 +91,7 @@ public: const StoragePtr & table, const ASTPtr & query) = 0; - /// Delete the table from the database and return it. Delete the metadata. + /// Delete the table from the database. Delete the metadata. virtual void removeTable( const Context & context, const String & name) = 0; From 52fea90a86ef02233a836cd60566dc6eaaa28831 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Nov 2018 17:33:40 +0300 Subject: [PATCH 25/47] Removed useless code [#CLICKHOUSE-2] --- dbms/src/Client/Connection.cpp | 8 --- dbms/src/Client/Connection.h | 5 -- dbms/src/Client/MultiplexedConnections.cpp | 20 +------ dbms/src/Client/MultiplexedConnections.h | 14 +---- dbms/src/Core/Block.h | 18 ------ dbms/src/DataStreams/IBlockInputStream.h | 7 --- .../src/DataStreams/ParallelInputsProcessor.h | 27 ++------- .../DataStreams/RemoteBlockInputStream.cpp | 9 +-- dbms/src/DataStreams/RemoteBlockInputStream.h | 9 --- dbms/src/DataStreams/UnionBlockInputStream.h | 58 ++----------------- dbms/src/DataStreams/tests/union_stream2.cpp | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 2 +- .../InterpreterSelectWithUnionQuery.cpp | 2 +- dbms/src/Storages/Kafka/StorageKafka.cpp | 2 +- 14 files changed, 21 insertions(+), 162 deletions(-) diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 2c704e0aa54..66bc04dcaf4 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -729,14 +729,6 @@ BlockStreamProfileInfo Connection::receiveProfileInfo() return profile_info; } -void Connection::fillBlockExtraInfo(BlockExtraInfo & info) const -{ - info.is_valid = true; - info.host = host; - info.resolved_address = getResolvedAddress().toString(); - info.port = port; - info.user = user; -} void Connection::throwUnexpectedPacket(UInt64 packet_type, const char * expected) const { diff --git a/dbms/src/Client/Connection.h b/dbms/src/Client/Connection.h index d8229fc3463..e3553e2a5cb 100644 --- a/dbms/src/Client/Connection.h +++ b/dbms/src/Client/Connection.h @@ -159,11 +159,6 @@ public: */ void disconnect(); - /** Fill in the information that is needed when getting the block for some tasks - * (so far only for a DESCRIBE TABLE query with Distributed tables). - */ - void fillBlockExtraInfo(BlockExtraInfo & info) const; - size_t outBytesCount() const { return out ? out->count() : 0; } size_t inBytesCount() const { return in ? in->count() : 0; } diff --git a/dbms/src/Client/MultiplexedConnections.cpp b/dbms/src/Client/MultiplexedConnections.cpp index 5228f9b85b3..62312eb5646 100644 --- a/dbms/src/Client/MultiplexedConnections.cpp +++ b/dbms/src/Client/MultiplexedConnections.cpp @@ -26,7 +26,7 @@ MultiplexedConnections::MultiplexedConnections(Connection & connection, const Se MultiplexedConnections::MultiplexedConnections( std::vector && connections, - const Settings & settings_, const ThrottlerPtr & throttler, bool append_extra_info) + const Settings & settings_, const ThrottlerPtr & throttler) : settings(settings_) { /// If we didn't get any connections from pool and getMany() did not throw exceptions, this means that @@ -48,9 +48,6 @@ MultiplexedConnections::MultiplexedConnections( } active_connection_count = connections.size(); - - if (append_extra_info) - block_extra_info = std::make_unique(); } void MultiplexedConnections::sendExternalTablesData(std::vector & data) @@ -126,24 +123,9 @@ Connection::Packet MultiplexedConnections::receivePacket() { std::lock_guard lock(cancel_mutex); Connection::Packet packet = receivePacketUnlocked(); - if (block_extra_info) - { - if (packet.type == Protocol::Server::Data) - current_connection->fillBlockExtraInfo(*block_extra_info); - else - block_extra_info->is_valid = false; - } return packet; } -BlockExtraInfo MultiplexedConnections::getBlockExtraInfo() const -{ - if (!block_extra_info) - throw Exception("MultiplexedConnections object not configured for block extra info support", - ErrorCodes::LOGICAL_ERROR); - return *block_extra_info; -} - void MultiplexedConnections::disconnect() { std::lock_guard lock(cancel_mutex); diff --git a/dbms/src/Client/MultiplexedConnections.h b/dbms/src/Client/MultiplexedConnections.h index 7955fb4d821..074a8c8d981 100644 --- a/dbms/src/Client/MultiplexedConnections.h +++ b/dbms/src/Client/MultiplexedConnections.h @@ -21,12 +21,10 @@ public: /// Accepts ready connection. MultiplexedConnections(Connection & connection, const Settings & settings_, const ThrottlerPtr & throttler_); - /** Accepts a vector of connections to replicas of one shard already taken from pool. - * If the append_extra_info flag is set, additional information appended to each received block. - */ + /// Accepts a vector of connections to replicas of one shard already taken from pool. MultiplexedConnections( - std::vector && connections, - const Settings & settings_, const ThrottlerPtr & throttler_, bool append_extra_info); + std::vector && connections, + const Settings & settings_, const ThrottlerPtr & throttler_); /// Send all content of external tables to replicas. void sendExternalTablesData(std::vector & data); @@ -42,9 +40,6 @@ public: /// Get packet from any replica. Connection::Packet receivePacket(); - /// Get information about the last received packet. - BlockExtraInfo getBlockExtraInfo() const; - /// Break all active connections. void disconnect(); @@ -99,11 +94,8 @@ private: /// Connection that received last block. Connection * current_connection = nullptr; - /// Information about the last received block, if supported. - std::unique_ptr block_extra_info; bool sent_query = false; - bool cancelled = false; /// A mutex for the sendCancel function to execute safely diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index fa0038e41e8..690f9720af1 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -151,22 +151,4 @@ void assertBlocksHaveEqualStructure(const Block & lhs, const Block & rhs, const /// Calculate difference in structure of blocks and write description into output strings. NOTE It doesn't compare values of constant columns. void getBlocksDifference(const Block & lhs, const Block & rhs, std::string & out_lhs_diff, std::string & out_rhs_diff); - -/** Additional data to the blocks. They are only needed for a query - * DESCRIBE TABLE with Distributed tables. - */ -struct BlockExtraInfo -{ - BlockExtraInfo() {} - operator bool() const { return is_valid; } - bool operator!() const { return !is_valid; } - - std::string host; - std::string resolved_address; - std::string user; - UInt16 port = 0; - - bool is_valid = false; -}; - } diff --git a/dbms/src/DataStreams/IBlockInputStream.h b/dbms/src/DataStreams/IBlockInputStream.h index b0f2d269f56..3e7c59ff78f 100644 --- a/dbms/src/DataStreams/IBlockInputStream.h +++ b/dbms/src/DataStreams/IBlockInputStream.h @@ -63,13 +63,6 @@ public: */ virtual Block read() = 0; - /** Get information about the last block received. - */ - virtual BlockExtraInfo getBlockExtraInfo() const - { - throw Exception("Method getBlockExtraInfo is not supported by the data stream " + getName(), ErrorCodes::NOT_IMPLEMENTED); - } - /** Read something before starting all data or after the end of all data. * In the `readSuffix` function, you can implement a finalization that can lead to an exception. * readPrefix() must be called before the first call to read(). diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index 500561e0282..3a6ed25f5c8 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -39,23 +39,12 @@ namespace CurrentMetrics namespace DB { -/** Union mode. - */ -enum class StreamUnionMode -{ - Basic = 0, /// take out blocks - ExtraInfo /// take out blocks + additional information -}; - /// Example of the handler. struct ParallelInputsHandler { /// Processing the data block. void onBlock(Block & /*block*/, size_t /*thread_num*/) {} - /// Processing the data block + additional information. - void onBlock(Block & /*block*/, BlockExtraInfo & /*extra_info*/, size_t /*thread_num*/) {} - /// Called for each thread, when the thread has nothing else to do. /// Due to the fact that part of the sources has run out, and now there are fewer sources left than streams. /// Called if the `onException` method does not throw an exception; is called before the `onFinish` method. @@ -70,7 +59,7 @@ struct ParallelInputsHandler }; -template +template class ParallelInputsProcessor { public: @@ -183,15 +172,9 @@ private: InputData(const BlockInputStreamPtr & in_, size_t i_) : in(in_), i(i_) {} }; - void publishPayload(BlockInputStreamPtr & stream, Block & block, size_t thread_num) + void publishPayload(Block & block, size_t thread_num) { - if constexpr (mode == StreamUnionMode::Basic) - handler.onBlock(block, thread_num); - else - { - BlockExtraInfo extra_info = stream->getBlockExtraInfo(); - handler.onBlock(block, extra_info, thread_num); - } + handler.onBlock(block, thread_num); } void thread(ThreadGroupStatusPtr thread_group, size_t thread_num) @@ -249,7 +232,7 @@ private: { additional_input_at_end->readPrefix(); while (Block block = additional_input_at_end->read()) - publishPayload(additional_input_at_end, block, thread_num); + publishPayload(block, thread_num); } catch (...) { @@ -312,7 +295,7 @@ private: break; if (block) - publishPayload(input.in, block, thread_num); + publishPayload(block, thread_num); } } } diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.cpp b/dbms/src/DataStreams/RemoteBlockInputStream.cpp index 670a70ad6bb..84d5a5ad000 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockInputStream.cpp @@ -45,7 +45,7 @@ RemoteBlockInputStream::RemoteBlockInputStream( create_multiplexed_connections = [this, connections, throttler]() mutable { return std::make_unique( - std::move(connections), context.getSettingsRef(), throttler, append_extra_info); + std::move(connections), context.getSettingsRef(), throttler); }; } @@ -74,7 +74,7 @@ RemoteBlockInputStream::RemoteBlockInputStream( connections = pool->getMany(¤t_settings, pool_mode); return std::make_unique( - std::move(connections), current_settings, throttler, append_extra_info); + std::move(connections), current_settings, throttler); }; } @@ -88,11 +88,6 @@ RemoteBlockInputStream::~RemoteBlockInputStream() multiplexed_connections->disconnect(); } -void RemoteBlockInputStream::appendExtraInfo() -{ - append_extra_info = true; -} - void RemoteBlockInputStream::readPrefix() { if (!sent_query) diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.h b/dbms/src/DataStreams/RemoteBlockInputStream.h index fd50735516c..9eb59ea60a9 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.h +++ b/dbms/src/DataStreams/RemoteBlockInputStream.h @@ -51,9 +51,6 @@ public: void setMainTable(QualifiedTableName main_table_) { main_table = std::move(main_table_); } - /// Besides blocks themself, get blocks' extra info - void appendExtraInfo(); - /// Sends query (initiates calculation) before read() void readPrefix() override; @@ -66,11 +63,6 @@ public: String getName() const override { return "Remote"; } - BlockExtraInfo getBlockExtraInfo() const override - { - return multiplexed_connections->getBlockExtraInfo(); - } - Block getHeader() const override { return header; } protected: @@ -143,7 +135,6 @@ private: */ std::atomic got_unknown_packet_from_replica { false }; - bool append_extra_info = false; PoolMode pool_mode = PoolMode::GET_MANY; std::optional main_table; diff --git a/dbms/src/DataStreams/UnionBlockInputStream.h b/dbms/src/DataStreams/UnionBlockInputStream.h index 0a2be37da8b..05807aa88d6 100644 --- a/dbms/src/DataStreams/UnionBlockInputStream.h +++ b/dbms/src/DataStreams/UnionBlockInputStream.h @@ -19,12 +19,8 @@ namespace ErrorCodes namespace { -template -struct OutputData; - /// A block or an exception. -template <> -struct OutputData +struct OutputData { Block block; std::exception_ptr exception; @@ -34,19 +30,6 @@ struct OutputData OutputData(std::exception_ptr & exception_) : exception(exception_) {} }; -/// Block + additional information or an exception. -template <> -struct OutputData -{ - Block block; - BlockExtraInfo extra_info; - std::exception_ptr exception; - - OutputData() {} - OutputData(Block & block_, BlockExtraInfo & extra_info_) : block(block_), extra_info(extra_info_) {} - OutputData(std::exception_ptr & exception_) : exception(exception_) {} -}; - } /** Merges several sources into one. @@ -58,20 +41,12 @@ struct OutputData * - with the help of ParallelInputsProcessor in several threads it takes out blocks from the sources; * - the completed blocks are added to a limited queue of finished blocks; * - the main thread takes out completed blocks from the queue of finished blocks; - * - if the StreamUnionMode::ExtraInfo mode is specified, in addition to the UnionBlockInputStream - * extracts blocks information; In this case all sources should support such mode. */ - -template class UnionBlockInputStream final : public IProfilingBlockInputStream { public: using ExceptionCallback = std::function; -private: - using Self = UnionBlockInputStream; - -public: UnionBlockInputStream(BlockInputStreams inputs, BlockInputStreamPtr additional_input_at_end, size_t max_threads, ExceptionCallback exception_callback_ = ExceptionCallback()) : output_queue(std::min(inputs.size(), max_threads)), @@ -125,11 +100,6 @@ public: processor.cancel(kill); } - BlockExtraInfo getBlockExtraInfo() const override - { - return doGetBlockExtraInfo(); - } - Block getHeader() const override { return children.at(0)->getHeader(); } protected: @@ -146,7 +116,7 @@ protected: /** Let's read everything up to the end, so that ParallelInputsProcessor is not blocked when trying to insert into the queue. * Maybe there is an exception in the queue. */ - OutputData res; + OutputData res; while (true) { //std::cerr << "popping\n"; @@ -230,20 +200,9 @@ protected: } private: - BlockExtraInfo doGetBlockExtraInfo() const - { - if constexpr (mode == StreamUnionMode::ExtraInfo) - return received_payload.extra_info; - else - throw Exception("Method getBlockExtraInfo is not supported for mode StreamUnionMode::Basic", - ErrorCodes::NOT_IMPLEMENTED); - } - -private: - using Payload = OutputData; + using Payload = OutputData; using OutputQueue = ConcurrentBoundedQueue; -private: /** The queue of the finished blocks. Also, you can put an exception instead of a block. * When data is run out, an empty block is inserted into the queue. * Sooner or later, an empty block is always inserted into the queue (even after exception or query cancellation). @@ -254,18 +213,13 @@ private: struct Handler { - Handler(Self & parent_) : parent(parent_) {} + Handler(UnionBlockInputStream & parent_) : parent(parent_) {} void onBlock(Block & block, size_t /*thread_num*/) { parent.output_queue.push(Payload(block)); } - void onBlock(Block & block, BlockExtraInfo & extra_info, size_t /*thread_num*/) - { - parent.output_queue.push(Payload(block, extra_info)); - } - void onFinish() { parent.output_queue.push(Payload()); @@ -287,11 +241,11 @@ private: parent.cancel(false); /// Does not throw exceptions. } - Self & parent; + UnionBlockInputStream & parent; }; Handler handler; - ParallelInputsProcessor processor; + ParallelInputsProcessor processor; ExceptionCallback exception_callback; diff --git a/dbms/src/DataStreams/tests/union_stream2.cpp b/dbms/src/DataStreams/tests/union_stream2.cpp index 377dd76bf2e..284e66b91a9 100644 --- a/dbms/src/DataStreams/tests/union_stream2.cpp +++ b/dbms/src/DataStreams/tests/union_stream2.cpp @@ -40,7 +40,7 @@ try for (size_t i = 0, size = streams.size(); i < size; ++i) streams[i] = std::make_shared(streams[i]); - BlockInputStreamPtr stream = std::make_shared>(streams, nullptr, settings.max_threads); + BlockInputStreamPtr stream = std::make_shared(streams, nullptr, settings.max_threads); stream = std::make_shared(stream, 10, 0); WriteBufferFromFileDescriptor wb(STDERR_FILENO); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 2a139edce9f..c48b7359751 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1309,7 +1309,7 @@ void InterpreterSelectQuery::executeUnion(Pipeline & pipeline) { unifyStreams(pipeline); - pipeline.firstStream() = std::make_shared>(pipeline.streams, pipeline.stream_with_non_joined_data, max_streams); + pipeline.firstStream() = std::make_shared(pipeline.streams, pipeline.stream_with_non_joined_data, max_streams); pipeline.stream_with_non_joined_data = nullptr; pipeline.streams.resize(1); } diff --git a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 44936397e20..7bc99ae8b9a 100644 --- a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -213,7 +213,7 @@ BlockIO InterpreterSelectWithUnionQuery::execute() } else { - result_stream = std::make_shared>(nested_streams, nullptr, settings.max_threads); + result_stream = std::make_shared(nested_streams, nullptr, settings.max_threads); nested_streams.clear(); } diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 03f45382da4..3d15259efcd 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -530,7 +530,7 @@ bool StorageKafka::streamToViews() // Join multiple streams if necessary BlockInputStreamPtr in; if (streams.size() > 1) - in = std::make_shared>(streams, nullptr, streams.size()); + in = std::make_shared(streams, nullptr, streams.size()); else in = streams[0]; From 7e9d053d5d40d35c750a6dc15e8ebcf2a7d1708e Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 28 Nov 2018 17:41:51 +0300 Subject: [PATCH 26/47] Benchmark: use client config for ssl options --- dbms/programs/benchmark/Benchmark.cpp | 23 +++++++++++-- dbms/programs/benchmark/CMakeLists.txt | 2 +- dbms/programs/client/Client.cpp | 18 ++-------- dbms/programs/performance-test/CMakeLists.txt | 2 +- dbms/src/Common/Config/configReadClient.cpp | 34 +++++++++++++++++++ dbms/src/Common/Config/configReadClient.h | 9 +++++ 6 files changed, 67 insertions(+), 21 deletions(-) create mode 100644 dbms/src/Common/Config/configReadClient.cpp create mode 100644 dbms/src/Common/Config/configReadClient.h diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 01b205e4f0f..901b51ac9ac 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -31,6 +31,7 @@ #include #include #include +#include /** A tool for evaluating ClickHouse performance. @@ -46,7 +47,7 @@ namespace ErrorCodes extern const int EMPTY_DATA_PASSED; } -class Benchmark +class Benchmark : public Poco::Util::Application { public: Benchmark(unsigned concurrency_, double delay_, @@ -75,13 +76,28 @@ public: else throw Exception("Unknown query processing stage: " + stage, ErrorCodes::BAD_ARGUMENTS); + } + + void initialize(Poco::Util::Application & self) + { + Poco::Util::Application::initialize(self); + std::string home_path; + const char * home_path_cstr = getenv("HOME"); + if (home_path_cstr) + home_path = home_path_cstr; + + configReadClient(config(), home_path); + } + + int main(const std::vector < std::string > & args) + { if (!json_path.empty() && Poco::File(json_path).exists()) /// Clear file with previous results { Poco::File(json_path).remove(); } readQueries(); - run(); + runBenchmark(); } private: @@ -220,7 +236,7 @@ private: return true; } - void run() + void runBenchmark() { pcg64 generator(randomSeed()); std::uniform_int_distribution distribution(0, queries.size() - 1); @@ -482,6 +498,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) options["json"].as(), ConnectionTimeouts::getTCPTimeoutsWithoutFailover(settings), settings); + return benchmark.run(); } catch (...) { diff --git a/dbms/programs/benchmark/CMakeLists.txt b/dbms/programs/benchmark/CMakeLists.txt index c41c46edeb8..af11c600b2d 100644 --- a/dbms/programs/benchmark/CMakeLists.txt +++ b/dbms/programs/benchmark/CMakeLists.txt @@ -1,5 +1,5 @@ add_library (clickhouse-benchmark-lib ${LINK_MODE} Benchmark.cpp) -target_link_libraries (clickhouse-benchmark-lib PRIVATE clickhouse-client-lib clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries (clickhouse-benchmark-lib PRIVATE clickhouse_aggregate_functions clickhouse-client-lib clickhouse_common_config clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) target_include_directories (clickhouse-benchmark-lib SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR}) if (CLICKHOUSE_SPLIT_BINARY) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index ee57b322b94..1b7e48ac55f 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -59,6 +59,7 @@ #include #include #include +#include #if USE_READLINE #include "Suggest.h" // Y_IGNORE @@ -206,22 +207,7 @@ private: if (home_path_cstr) home_path = home_path_cstr; - std::string config_path; - if (config().has("config-file")) - config_path = config().getString("config-file"); - else if (Poco::File("./clickhouse-client.xml").exists()) - config_path = "./clickhouse-client.xml"; - else if (!home_path.empty() && Poco::File(home_path + "/.clickhouse-client/config.xml").exists()) - config_path = home_path + "/.clickhouse-client/config.xml"; - else if (Poco::File("/etc/clickhouse-client/config.xml").exists()) - config_path = "/etc/clickhouse-client/config.xml"; - - if (!config_path.empty()) - { - ConfigProcessor config_processor(config_path); - auto loaded_config = config_processor.loadConfig(); - config().add(loaded_config.configuration); - } + configReadClient(config(), home_path); context.setApplicationType(Context::ApplicationType::CLIENT); diff --git a/dbms/programs/performance-test/CMakeLists.txt b/dbms/programs/performance-test/CMakeLists.txt index adad45025c3..f1a08172009 100644 --- a/dbms/programs/performance-test/CMakeLists.txt +++ b/dbms/programs/performance-test/CMakeLists.txt @@ -1,5 +1,5 @@ add_library (clickhouse-performance-test-lib ${LINK_MODE} PerformanceTest.cpp) -target_link_libraries (clickhouse-performance-test-lib PRIVATE dbms clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries (clickhouse-performance-test-lib PRIVATE dbms clickhouse_common_io clickhouse_common_config ${Boost_PROGRAM_OPTIONS_LIBRARY}) target_include_directories (clickhouse-performance-test-lib SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR}) if (CLICKHOUSE_SPLIT_BINARY) diff --git a/dbms/src/Common/Config/configReadClient.cpp b/dbms/src/Common/Config/configReadClient.cpp new file mode 100644 index 00000000000..9ab9439099d --- /dev/null +++ b/dbms/src/Common/Config/configReadClient.cpp @@ -0,0 +1,34 @@ +#include "configReadClient.h" + +#include +#include +#include +#include "ConfigProcessor.h" + +namespace DB +{ +bool configReadClient(Poco::Util::LayeredConfiguration & config, const std::string & home_path) +{ + std::string config_path; + if (config.has("config-file")) + config_path = config.getString("config-file"); + else if (Poco::File("./clickhouse-client.xml").exists()) + config_path = "./clickhouse-client.xml"; + else if (!home_path.empty() && Poco::File(home_path + "/.clickhouse-client/config.xml").exists()) + config_path = home_path + "/.clickhouse-client/config.xml"; + else if (Poco::File("/etc/clickhouse-client/config.xml").exists()) + config_path = "/etc/clickhouse-client/config.xml"; + +DUMP(config_path); + + if (!config_path.empty()) + { + ConfigProcessor config_processor(config_path); + auto loaded_config = config_processor.loadConfig(); + config.add(loaded_config.configuration); + return true; + } + return false; +} + +} \ No newline at end of file diff --git a/dbms/src/Common/Config/configReadClient.h b/dbms/src/Common/Config/configReadClient.h new file mode 100644 index 00000000000..4d7fa9f2ccb --- /dev/null +++ b/dbms/src/Common/Config/configReadClient.h @@ -0,0 +1,9 @@ +#pragma once +#include + +namespace Poco { class Logger; namespace Util { class LayeredConfiguration; } } + +namespace DB +{ +bool configReadClient(Poco::Util::LayeredConfiguration & config, const std::string & home_path); +} From a991781937b8a43587ed39627e0569854ff5b0f5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 28 Nov 2018 17:45:01 +0300 Subject: [PATCH 27/47] Update configReadClient.cpp --- dbms/src/Common/Config/configReadClient.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/dbms/src/Common/Config/configReadClient.cpp b/dbms/src/Common/Config/configReadClient.cpp index 9ab9439099d..01ad421cc2b 100644 --- a/dbms/src/Common/Config/configReadClient.cpp +++ b/dbms/src/Common/Config/configReadClient.cpp @@ -19,8 +19,6 @@ bool configReadClient(Poco::Util::LayeredConfiguration & config, const std::stri else if (Poco::File("/etc/clickhouse-client/config.xml").exists()) config_path = "/etc/clickhouse-client/config.xml"; -DUMP(config_path); - if (!config_path.empty()) { ConfigProcessor config_processor(config_path); @@ -30,5 +28,4 @@ DUMP(config_path); } return false; } - -} \ No newline at end of file +} From a4fd7f4eb332dc2746d49a529e4b9e337dcd2bbd Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 28 Nov 2018 17:45:58 +0300 Subject: [PATCH 28/47] Update configReadClient.h --- dbms/src/Common/Config/configReadClient.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Common/Config/configReadClient.h b/dbms/src/Common/Config/configReadClient.h index 4d7fa9f2ccb..057c3346c8b 100644 --- a/dbms/src/Common/Config/configReadClient.h +++ b/dbms/src/Common/Config/configReadClient.h @@ -5,5 +5,6 @@ namespace Poco { class Logger; namespace Util { class LayeredConfiguration; } } namespace DB { +/// Read configuration files related to clickhouse-client like applications. Returns true if any configuration files were read. bool configReadClient(Poco::Util::LayeredConfiguration & config, const std::string & home_path); } From 4c527b84d24f5625d385f4cb7dab2e221b29d6c9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Nov 2018 17:50:02 +0300 Subject: [PATCH 29/47] Fixed build #3688 --- dbms/programs/benchmark/Benchmark.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 901b51ac9ac..81c3583b3a9 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -89,15 +89,14 @@ public: configReadClient(config(), home_path); } - int main(const std::vector < std::string > & args) + int main(const std::vector &) { if (!json_path.empty() && Poco::File(json_path).exists()) /// Clear file with previous results - { Poco::File(json_path).remove(); - } readQueries(); runBenchmark(); + return 0; } private: From 32baabd9b002b4aabba920fd18cb96c8b24920bc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Nov 2018 18:27:23 +0300 Subject: [PATCH 30/47] Fixed bad code #3688 --- dbms/programs/benchmark/Benchmark.cpp | 2 +- .../performance-test/PerformanceTest.cpp | 36 ++++++++++--------- 2 files changed, 21 insertions(+), 17 deletions(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 81c3583b3a9..6b9494532c3 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -447,7 +447,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ("json", value()->default_value(""), "write final report to specified file in JSON format") ("host,h", value()->default_value("localhost"), "") ("port", value()->default_value(9000), "") - ("secure", value()->default_value(false), "use secure connection") + ("secure", "Use TLS connection") ("user", value()->default_value("default"), "") ("password", value()->default_value(""), "") ("database", value()->default_value("default"), "") diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index 6e6cedbef9e..d010ac707f7 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -494,11 +494,11 @@ public: PerformanceTest(const String & host_, const UInt16 port_, - const bool & secure_, + const bool secure_, const String & default_database_, const String & user_, const String & password_, - const bool & lite_output_, + const bool lite_output_, const String & profiles_file_, Strings && input_files_, Strings && tests_tags_, @@ -1384,24 +1384,28 @@ try using Strings = std::vector; boost::program_options::options_description desc("Allowed options"); - desc.add_options()("help", "produce help message")("lite", "use lite version of output")( - "profiles-file", value()->default_value(""), "Specify a file with global profiles")( - "host,h", value()->default_value("localhost"), "") + desc.add_options() + ("help", "produce help message") + ("lite", "use lite version of output") + ("profiles-file", value()->default_value(""), "Specify a file with global profiles") + ("host,h", value()->default_value("localhost"), "") ("port", value()->default_value(9000), "") - ("secure", value()->default_value(false), "use secure connection") - ( - "database", value()->default_value("default"), "")("user", value()->default_value("default"), "")( - "password", value()->default_value(""), "")("tags", value()->multitoken(), "Run only tests with tag")( - "skip-tags", value()->multitoken(), "Do not run tests with tag")("names", - value()->multitoken(), - "Run tests with specific name")("skip-names", value()->multitoken(), "Do not run tests with name")( - "names-regexp", value()->multitoken(), "Run tests with names matching regexp")("skip-names-regexp", - value()->multitoken(), - "Do not run tests with names matching regexp")("recursive,r", "Recurse in directories to find all xml's"); + ("secure", "Use TLS connection") + ("database", value()->default_value("default"), "") + ("user", value()->default_value("default"), "") + ("password", value()->default_value(""), "") + ("tags", value()->multitoken(), "Run only tests with tag") + ("skip-tags", value()->multitoken(), "Do not run tests with tag") + ("names", value()->multitoken(), "Run tests with specific name") + ("skip-names", value()->multitoken(), "Do not run tests with name") + ("names-regexp", value()->multitoken(), "Run tests with names matching regexp") + ("skip-names-regexp", value()->multitoken(), "Do not run tests with names matching regexp") + ("recursive,r", "Recurse in directories to find all xml's"); /// These options will not be displayed in --help boost::program_options::options_description hidden("Hidden options"); - hidden.add_options()("input-files", value>(), ""); + hidden.add_options() + ("input-files", value>(), ""); /// But they will be legit, though. And they must be given without name boost::program_options::positional_options_description positional; From 42631ba857ba43896a73e80a720802ecbcb3654a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Nov 2018 18:31:09 +0300 Subject: [PATCH 31/47] Fixed bad code #3688 --- dbms/programs/benchmark/Benchmark.cpp | 2 +- dbms/programs/performance-test/PerformanceTest.cpp | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 6b9494532c3..d043a91b8b4 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -486,7 +486,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) options["delay"].as(), options["host"].as(), options["port"].as(), - options["secure"].as(), + options.count("secure"), options["database"].as(), options["user"].as(), options["password"].as(), diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index d010ac707f7..4bdb1d64096 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -1482,9 +1482,10 @@ try DB::UseSSL use_ssl; - DB::PerformanceTest performanceTest(options["host"].as(), + DB::PerformanceTest performanceTest( + options["host"].as(), options["port"].as(), - options["secure"].as(), + options.count("secure"), options["database"].as(), options["user"].as(), options["password"].as(), From 2cc82f5cbe266421cd4c1165286c2c47e5ffcb15 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Nov 2018 19:27:39 +0300 Subject: [PATCH 32/47] Enable jemalloc profiling for debug builds [#CLICKHOUSE-2] --- contrib/jemalloc-cmake/CMakeLists.txt | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 696ea5fee10..94b84884f41 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -52,5 +52,11 @@ target_include_directories(jemalloc PRIVATE target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_NO_PRIVATE_NAMESPACE) if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") - target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_DEBUG=1) + target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_DEBUG=1 -DJEMALLOC_PROF=1) + + if (USE_UNWIND) + target_compile_definitions (jemalloc PRIVATE -DJEMALLOC_PROF_LIBUNWIND=1) + target_include_directories (jemalloc BEFORE PRIVATE ${UNWIND_INCLUDE_DIR}) + target_link_libraries (jemalloc PRIVATE ${UNWIND_LIBRARY}) + endif () endif () From f615b062d88e429d61498671b0a050cd776acc30 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Nov 2018 19:29:56 +0300 Subject: [PATCH 33/47] Fixed build [#CLICKHOUSE-2] --- dbms/src/DataStreams/UnionBlockInputStream.h | 28 +++++++++----------- 1 file changed, 12 insertions(+), 16 deletions(-) diff --git a/dbms/src/DataStreams/UnionBlockInputStream.h b/dbms/src/DataStreams/UnionBlockInputStream.h index 05807aa88d6..da24c9b131a 100644 --- a/dbms/src/DataStreams/UnionBlockInputStream.h +++ b/dbms/src/DataStreams/UnionBlockInputStream.h @@ -16,22 +16,6 @@ namespace ErrorCodes } -namespace -{ - -/// A block or an exception. -struct OutputData -{ - Block block; - std::exception_ptr exception; - - OutputData() {} - OutputData(Block & block_) : block(block_) {} - OutputData(std::exception_ptr & exception_) : exception(exception_) {} -}; - -} - /** Merges several sources into one. * Blocks from different sources are interleaved with each other in an arbitrary way. * You can specify the number of threads (max_threads), @@ -44,6 +28,18 @@ struct OutputData */ class UnionBlockInputStream final : public IProfilingBlockInputStream { +private: + /// A block or an exception. + struct OutputData + { + Block block; + std::exception_ptr exception; + + OutputData() {} + OutputData(Block & block_) : block(block_) {} + OutputData(std::exception_ptr & exception_) : exception(exception_) {} + }; + public: using ExceptionCallback = std::function; From 25c98efa6d56dda1a34f6df636698acd6a66a056 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Nov 2018 19:51:43 +0300 Subject: [PATCH 34/47] Added instruction [#CLICKHOUSE-2] --- .../instructions/jemalloc_memory_profile.txt | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) create mode 100644 dbms/tests/instructions/jemalloc_memory_profile.txt diff --git a/dbms/tests/instructions/jemalloc_memory_profile.txt b/dbms/tests/instructions/jemalloc_memory_profile.txt new file mode 100644 index 00000000000..e4119d27c09 --- /dev/null +++ b/dbms/tests/instructions/jemalloc_memory_profile.txt @@ -0,0 +1,23 @@ +# Generate "jeprof" tool. + +cd contrib/jemalloc +./autogen.sh + +# The tool is in /bin directory. Copy it somewhere or add it to PATH. + +cp bin/jeprof ... + +# Build ClickHouse in debug mode. + +# Set environment variable to enable profiling. + +export MALLOC_CONF="prof:true,prof_accum:true,lg_prof_interval:28" + +# Run clickhouse-server. + +# ClickHouse will generate profile dumps with names like jeprof* +# Use jeprof tool to analyze them: + +./jeprof ./clickhouse-server jeprof.4017.0.f.heap + +# It will spend a few minutes before displaying prompt. From 540a99c93a69c926d22a45dc72617ff8c66a82e1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 28 Nov 2018 19:52:46 +0300 Subject: [PATCH 35/47] Fix engine parsing in create query. --- dbms/src/Parsers/ParserCreateQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index 9b2a2fc931c..fb83572b18c 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -287,7 +287,8 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } /// Optional - ENGINE can be specified. - storage_p.parse(pos, storage, expected); + if (!storage) + storage_p.parse(pos, storage, expected); } } } From 14ff6ae6d4c51fa41952a25b9ff1e02e6b7c5670 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 28 Nov 2018 20:13:38 +0300 Subject: [PATCH 36/47] Added test. --- .../00763_create_query_as_table_engine_bug.reference | 1 + .../00763_create_query_as_table_engine_bug.sql | 8 ++++++++ 2 files changed, 9 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00763_create_query_as_table_engine_bug.reference create mode 100644 dbms/tests/queries/0_stateless/00763_create_query_as_table_engine_bug.sql diff --git a/dbms/tests/queries/0_stateless/00763_create_query_as_table_engine_bug.reference b/dbms/tests/queries/0_stateless/00763_create_query_as_table_engine_bug.reference new file mode 100644 index 00000000000..366823b0b8b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00763_create_query_as_table_engine_bug.reference @@ -0,0 +1 @@ +Distributed diff --git a/dbms/tests/queries/0_stateless/00763_create_query_as_table_engine_bug.sql b/dbms/tests/queries/0_stateless/00763_create_query_as_table_engine_bug.sql new file mode 100644 index 00000000000..fbaa637fd50 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00763_create_query_as_table_engine_bug.sql @@ -0,0 +1,8 @@ +drop table if exists test.t; +drop table if exists test.td; +create table test.t (val UInt32) engine = MergeTree order by val; +create table test.td engine = Distributed(test_cluster, 'test', 't') as test.t; +select engine from system.tables where database = 'test' and name = 'td'; +drop table if exists test.t; +drop table if exists test.td; + From 97280bb36182ce2a353ab3edbd5673499819eba0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Nov 2018 21:28:30 +0300 Subject: [PATCH 37/47] Whitespace [#CLICKHOUSE-2] --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index c48b7359751..507cd353f4e 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -929,8 +929,7 @@ void InterpreterSelectQuery::executeFetchColumns( if (query_info.prewhere_info) pipeline.streams.back() = std::make_shared( pipeline.streams.back(), prewhere_info->prewhere_actions, - prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column - ); + prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column); } pipeline.transform([&](auto & stream) From b4d218a890eca962f133d3dfccda461dc3ff25ff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Nov 2018 21:40:02 +0300 Subject: [PATCH 38/47] Fixed race condition, part 1 (ad-hoc) [#CLICKHOUSE-2] --- dbms/src/Storages/StorageBuffer.cpp | 5 +++++ dbms/src/Storages/StorageMaterializedView.cpp | 13 +++++++++++-- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index c093641b09f..447852ab3b5 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -178,7 +178,12 @@ BlockInputStreams StorageBuffer::read( LOG_WARNING(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) << " has no common columns with block in buffer. Block of data is skipped."); else + { + auto lock = destination->lockStructure(false, __PRETTY_FUNCTION__); streams_from_dst = destination->read(columns_intersection, query_info, context, processed_stage, max_block_size, num_streams); + for (auto & stream : streams_from_dst) + stream->addTableLock(lock); + } if (struct_mismatch && !streams_from_dst.empty()) { diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index f69efad04ae..ab691529ac9 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -185,12 +185,21 @@ BlockInputStreams StorageMaterializedView::read( const size_t max_block_size, const unsigned num_streams) { - return getTargetTable()->read(column_names, query_info, context, processed_stage, max_block_size, num_streams); + auto storage = getTargetTable(); + auto lock = storage->lockStructure(false, __PRETTY_FUNCTION__); + auto streams = storage->read(column_names, query_info, context, processed_stage, max_block_size, num_streams); + for (auto & stream : streams) + stream->addTableLock(lock); + return streams; } BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const Settings & settings) { - return getTargetTable()->write(query, settings); + auto storage = getTargetTable(); + auto lock = storage->lockStructure(true, __PRETTY_FUNCTION__); + auto stream = storage->write(query, settings); + stream->addTableLock(lock); + return stream; } From 46d8f2f985ee8195895afa439fea67f3f1b3bd1c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Nov 2018 22:09:17 +0300 Subject: [PATCH 39/47] Added test #3694 --- .../0_stateless/00763_lock_buffer.reference | 10374 ++++++++++++++++ .../queries/0_stateless/00763_lock_buffer.sh | 29 + 2 files changed, 10403 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00763_lock_buffer.reference create mode 100755 dbms/tests/queries/0_stateless/00763_lock_buffer.sh diff --git a/dbms/tests/queries/0_stateless/00763_lock_buffer.reference b/dbms/tests/queries/0_stateless/00763_lock_buffer.reference new file mode 100644 index 00000000000..effae2417a9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00763_lock_buffer.reference @@ -0,0 +1,10374 @@ +[milovidov-Pro-P30] 2018.11.28 22:08:15.804398 {b382b34d-e62b-4a0a-bb8b-5671b9d82a40} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:15.839436 {12df6433-eab4-4494-8647-1eee0dda68a3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:15.879572 {8117c51b-d65b-4ec1-b4b7-e9a77407ced2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:15.886553 {aa40f63b-8632-45a2-bc38-ef891efb1dff} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:15.924566 {f20c3ceb-5efb-4d69-96a0-d8eadf558a77} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:15.930785 {06b4ed5e-9851-4ce5-b545-04af59655729} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:15.967409 {a0a04a9d-5088-47e9-a3de-944a28746862} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:15.973558 {4b580694-b7a4-422e-bb3a-4b185ddf8ae8} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.010300 {9dbfaeff-08e5-47ed-8bbd-d9276446382c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.016618 {c994f4e5-ac82-49d6-90d2-a9c857a7fe61} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.052850 {1d26d531-2005-4f49-9f66-6962a3ebf796} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.059333 {eb95a219-c505-436b-93b9-ebbaf36ecbb0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.097461 {6276b2af-ce73-4ac4-826a-a73dc1a3eaf3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.105200 {0caa94fe-74b6-4471-b9ac-c7d85fe33007} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.143043 {93b81cb5-525b-4e94-86c0-d5919d7608ea} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.150518 {fd8c908a-e0b5-4fb2-b419-bd08c21c3c13} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.190703 {7f788212-7a46-4f3c-acb4-c583b28e9df2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.197392 {878132f9-5219-4aea-914b-0dbd7b58a764} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.233792 {8d04aba7-9125-467e-8ea6-0e16bc7bf7c6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.239797 {094bd4d8-0eac-4e84-8b3c-2ea07d3b4916} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.278886 {d599a104-eaeb-4b60-b603-12b65892dc38} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.285176 {69888f58-b324-43f8-80c5-b0b8e6561649} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.324470 {583d7fc9-db34-4550-81ae-6ff31a1d96b8} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.330644 {b518f495-9f63-4b92-aa39-75a24398866b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.369672 {28378d76-b475-49ee-8908-9be52469e0a1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.375686 {b0f79188-0e4f-4980-9d1f-78ce9db2f7cf} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.412233 {6fc614df-02d1-4c3a-aff1-f03bc3ef0aaa} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.418180 {54059a5c-7754-4552-b140-c0e4a86ef40e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.458263 {5fa1e884-db86-451a-80a0-d8274019c10c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.465598 {b5bec235-c220-4a4b-b5e4-2c57179c2e91} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.502333 {78149d68-ab22-4898-9596-15b207f99ef6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.509010 {6c533a36-c9fc-4c79-b334-9fdb066d9464} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.547343 {8f4e117f-c957-4955-a7a3-466a3df25a89} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.553354 {acada29b-8541-4054-bb97-bb35e88eab27} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.590795 {89266bb1-4e35-44ec-bf2d-9bc1c26f1a74} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.596932 {7b0f1862-2909-4b1a-b476-e169d65b5dfc} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.634903 {f3449d01-002e-46a5-9bd4-120777f6bfec} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.640730 {fe17c38b-0b23-4615-814f-7903fb2cde0c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.678091 {8c0bcdbb-ca41-4827-b170-cd933fb8b651} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.683945 {872c146c-d6bd-4cf1-b61d-0e26884097d1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.719134 {931a96e6-7973-403c-9ba9-951a22711f14} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.725387 {56c9ef93-b51e-49e1-9044-75a8e1adb420} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.761859 {4ab5c13c-b88f-47f0-b95e-0fe36678b000} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.769417 {585d204d-88ea-43f2-bbe3-31caa4626e4b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.805721 {692534b6-6558-4913-92e1-8aaefe56ed57} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.811425 {e027f56f-f8a1-4b24-8d49-6f4bd5b340ed} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.846068 {610c8387-3df4-424f-9c5e-9d7eb18debae} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.851569 {76a2eb5e-1370-4e12-8715-21750af16bed} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.887480 {86c45297-16ab-473d-b201-e2236e5b6116} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.893490 {9f2e0e03-8363-48b8-8829-d922084ba301} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.929084 {70ef1316-e73c-4ac8-b417-e53dd38722ea} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.936277 {5d2fa797-629c-4568-99b6-0e65ab27be7f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.972299 {f3a0a47b-2399-4022-9d54-f37af9eff412} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:16.978024 {5a74a293-7e13-41b9-a80d-f5a4e9b2e868} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.016729 {d92291d9-f4fc-4823-bbc6-71785fb5d07c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.022342 {30486679-6c7f-4877-88b3-f3242d483466} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.060484 {70852fb6-c64c-47e4-b1f4-e8d7f8432522} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.066117 {2fba7d1c-b08c-4a2b-bea4-9d4dd80e4975} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.102132 {c3918e15-7842-4887-bde0-1928a36bda2c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.107582 {97a724d7-97d7-4466-af61-23e3e34ba601} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.142054 {771188c8-f84f-4463-a559-1583432eba4b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.147795 {b09b2a35-a72a-4d3e-8d74-f09f57d449db} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.184394 {f982786e-131d-42e5-8214-ae3aab11328d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.217858 {38f72a08-ecd5-45ab-8899-ff2fbcec0579} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.223712 {742b544a-7c8d-4466-bf15-f9584991e311} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.257521 {5382a35e-3be7-4a30-9b7b-058a8601f264} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.262962 {4bfae50b-e211-4be2-9d76-d0aba72eb2eb} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.301359 {8c2b0b6f-0f3b-4c29-83db-6840b05f2837} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.307158 {a6dc969f-192c-46c2-ab0e-00856a63bcb3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.343061 {a129bb03-266b-4562-8a6d-00a0e73ee8f1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.350189 {22e13b6d-958b-4429-980a-9637f0d2f676} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.384017 {ae678170-bcc3-4679-b883-9715af969b78} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.389498 {f82bd3f7-1b76-44bc-af73-7dbaf1c699f0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.425316 {80c8d353-e8f9-423c-8a24-c3d374bb3145} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.431225 {df4e4518-3853-4528-b159-ecd9eda8ff09} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.465149 {5b535596-a956-43b1-a2f7-2da7c656b5f6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.470614 {8f63bd16-446a-4fae-8ad7-f7dee705a21a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.506173 {b833c309-001b-4a29-a16d-9486c6f9cd20} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.511528 {0268a36b-c2ab-4dbc-8382-23964f1259b1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.544800 {987ed9ac-be0a-4ef6-9171-0f50740ad8db} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.550231 {a3f8b19b-a42e-4e5b-a0f3-914421f11405} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.583151 {75233a53-9450-4cfb-9a2a-791ba27cd2d7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.588621 {579b7452-06f6-400a-b25b-201d81e8f705} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.626467 {99852f0e-0063-499a-8391-4f8b63a1d455} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.632010 {e8743ea8-ab0e-4f89-bff5-e07ae11c14d1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.667868 {6240085d-c1fb-4173-b9ea-a8435fe6a445} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.674577 {511fd4e2-d615-4413-a560-889ac496b886} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.709817 {dc2485cd-6028-4cbc-bf9c-7bc17393260e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.715129 {8c7fba23-6e43-4f9b-ad3e-3d2116eb1e08} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.748908 {e404b337-cd8a-4a35-8d56-750dd327d2be} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.754181 {6291c4ae-3d6e-496d-87f5-eed780a6a2b0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.788946 {8326522b-1a7d-45ae-ad79-97bf5f44c617} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.794666 {02a7b413-613b-42d8-ab7c-4342f2b6a669} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.829819 {40180165-0671-415a-b3ee-75711f5bcf3b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.835307 {08e18b86-0925-40ac-a73c-739d3758ffc1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.871530 {bb1b2073-4709-40ca-aed8-1eb0d5983cab} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.878301 {3630c665-036c-4b70-b9f0-3d99e79e983d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.913240 {96fda9b7-54f2-4b16-84b8-f551a5ec8abf} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.918661 {555cec54-7d2d-481e-994e-6ec08626cefd} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.952820 {6e9bb6ac-4d91-4fcc-b3cd-f0fc440f50d1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.958480 {85942327-3b97-475b-9b94-07fdea68e5fa} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.993680 {80aee0e2-7712-4c5e-a919-d9f0f64ca0da} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:17.999245 {cd705bb9-9402-4d61-b05a-d8142209b78d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.033388 {abaf0831-9199-4cb8-8010-8637caf73715} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.038605 {dfd8a3a0-5e50-4afe-ae4b-d1a2b88383f7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.073803 {b3737f44-61a2-44b1-9dae-3fff37021d32} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.078988 {573e82f0-3fa6-4ed5-a6bd-9836d4c376d2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.116072 {8005c879-14fc-4f4d-a65b-cdd8ccb95f02} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.121410 {04cc6056-7e88-425f-b5c0-ccd3f913b128} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.157133 {c0475c4c-86b2-4f77-90ea-e787beff8e79} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.162408 {88849363-2290-4e35-9bfc-4fa059991725} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.195365 {6dacc238-b4e6-4229-a124-5bdce40b2315} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.202200 {ecded22b-1ffa-4a09-9f98-65b60daed8f2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.239109 {85a22162-577b-436b-8ea5-5dd04b971362} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.244307 {1c5fbc39-87fc-4d1c-b0e9-f151f6e5320d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.277030 {d88bc3fa-0534-40f9-bd34-7f102702b971} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.282619 {fe704672-05a9-431b-9248-27ba2d50da2b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.316182 {fe0bcc92-42f2-471c-a54a-99bee226c12c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.321458 {d2a4cb2d-d450-419f-86eb-fc34543ad4e2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.354352 {a12c0c42-2a78-48ed-bfd6-9b0c8486b97f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.359477 {f73fd3f6-ada8-4d16-94b7-d682753e8814} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.392610 {e18daa32-8309-46ca-9b21-89d922f52df9} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.397918 {f518e531-736a-4811-855d-950ba52acd3d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.431508 {fbf7a0c3-ce49-4b92-8c0f-f68049d04242} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.438271 {f33025eb-42be-4d3c-b7a2-02f47668e987} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.471301 {88aae2db-b56f-4b8b-9545-123c5a22052a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.476369 {6739091d-fdd5-4a5b-ba7f-16b468222467} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.510199 {ff128f44-605b-49a2-863c-d45c557ab6ef} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.516160 {d1d3b6d0-7e38-473c-8455-e16fb059baae} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.549400 {142da820-42d7-4d60-8f90-a014b8d5978f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.554544 {d0bf63f6-9458-40fd-96e7-8430634896e7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.587420 {b7df241e-b337-4a9c-a400-694ece7745ac} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.592825 {d62c459f-a84a-406d-96dc-50615862df94} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.624438 {b1dd3f0a-d2c2-420a-89fe-739caffa972c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.629489 {a88a9512-0d41-4b74-b010-d316d099fc6c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.660508 {79e5a325-c92a-41db-9743-eba05c45bfa1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.665771 {f1cdea6e-d097-4633-a4dc-04542c1f9e01} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.699952 {94144751-cb9c-449b-8090-9cd064d4e10c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.704897 {85ae60e6-fd10-4abd-b42f-a29208b46b3b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.735563 {2a6c79f1-3839-46c0-b310-2850b7e23eba} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.740511 {a3aedcd6-49a7-4309-a191-ca698e6a419e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.771576 {b0932dd8-dce1-4339-9b7d-b8340b1874a0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.776862 {0e71cf19-2563-48ed-bf66-f15a309f29f5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.810621 {75dc62a3-8c9f-44e9-b194-233f7819bda8} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.815959 {eee05c7a-05ed-47e6-9032-7ae4cc27c262} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.851936 {f3391943-b4db-4fd2-bcc5-67c522672a2b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.857406 {1d40ead8-ed6f-4552-9e06-9889cb616e06} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.889979 {5b02c382-5c76-4fc4-b52f-a367dd2a34e2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.894979 {a4102dba-66e7-4c23-a79f-18372bc18eda} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.928024 {74a886a1-18a0-4da7-8179-b305e14306b4} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.932999 {89679622-084d-422a-a412-8da65ab4a152} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.967572 {c4d1efeb-abb5-4afc-bce9-f0a83ddc52f3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:18.972741 {ecfef9e5-7656-4440-93d7-6b0de9dc5936} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.005420 {16b6a55c-3f1a-40fe-9197-8ef1fee26ad6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.010876 {4bf80c2f-f584-4634-a4cb-afd2da5f17f2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.042372 {3f8516f7-1630-413b-9f1a-c6e10a3489d3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.047239 {416eff96-a99c-44d1-8a9c-43aea8620633} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.052187 {c8f87f7a-86dc-487d-a2e7-0d345426fba0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.084790 {4a5ea3c0-1d66-488f-9735-10e35c4fa20c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.091202 {ec69c65b-8666-4d5d-a9e0-cc5f1f9bc5d7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.124920 {6e993fe5-d826-4c93-9afb-cf56666d99de} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.129856 {bfc0f2b4-ed16-471b-94d7-a84c1e712700} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.163197 {934d9b62-9279-4e1a-ba5e-dc49b1537c41} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.167999 {ecc36b79-a499-43ed-8d11-d9246353a6a7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.199826 {41623caa-87c5-4c45-90e5-786eb64927d8} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.204546 {a3a8403a-1db4-4b1a-bcda-b3e6df050b7d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.235716 {58068b04-dd39-4efc-90c3-472a80c63581} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.242427 {08fe6947-b10c-4d2c-8f66-874867f96e26} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.281260 {998ed272-ce7f-4d73-8e8e-7337b83411e3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.286139 {0539dbe3-2fa1-42ec-ac7d-a6fc178e08e6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.318034 {30d2f17f-eb27-445d-aa94-4d1849cd17d4} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.324364 {14205a4d-d99c-4f36-be39-b92fffd1f4d1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.356909 {0b482022-ef79-49e9-a280-f7654a59b652} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.362271 {f483b92e-e257-4569-9908-be72d6467878} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.395056 {8690f680-f715-47be-8f60-bfc23073ccff} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.401412 {b00ecf03-4094-4d84-a999-e16bd3c955e6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.439565 {aaf84738-51d9-4664-b0d1-05eab0673069} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.444273 {f5942cd3-9ac8-44f0-bffa-649635c7f846} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.478090 {aed97b0a-37a1-4858-bbfa-431a5e8f7a8a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.483467 {18de9ba6-96a6-44af-9342-1841e273ad1b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.515387 {73491b80-9470-464e-ae2a-37dad11887a5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.520106 {b56c5c49-5af0-4329-b81f-cf0ee0a1807f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.550740 {c9fac8fd-b266-4612-a231-4e8a9dfc7c09} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.555584 {18f6af96-bcc5-4877-ba87-994492302d83} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.588382 {680e7447-65f1-4a34-b326-18d7eef0ac4d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.593073 {1c0d3ac2-de89-4fe6-94f5-1c31f68e9ae5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.625452 {0d8c5081-3a3b-4c0c-842c-04dbcd46796d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.631467 {943503e7-4311-4ecf-85d6-09103b6301b9} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.662721 {92b83212-21fb-4562-b152-3d83c1984ec2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.667347 {e9313bf0-74ba-419f-b388-f9f74b916d34} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.697209 {c1ae5516-ce7a-41bc-bc3a-4ed84cbeac31} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.701821 {169682d3-59ee-4315-a69d-d30a63484a2e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.732723 {dc58c84f-e2a7-487e-8635-f7b90e37e6b0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.737300 {abb290a9-2475-4f3e-92e7-210f422e6c35} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.768092 {022060f1-46a9-4c06-ab85-eb89228d5ac1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.772777 {c54a77a3-6bd3-4a1e-a463-8c1617fe1729} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.806728 {622acf90-b971-469e-a275-efaeb127d5fe} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.811228 {3ceaed60-d474-402f-9475-7656e201e555} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.843577 {be73cadf-ddec-4986-a225-bc3bfafd746e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.848357 {665df9d2-5875-48a7-a342-aff8c1ecd46c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.880431 {970e148e-5956-4b87-bce7-be4fbfdf1d9f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.885736 {27463163-3b99-4724-b6eb-e4273078d8a9} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.916999 {c1fa3ba7-9304-41f7-975c-907ddd2ce75d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.921450 {fad8c602-fad8-4247-9454-b36481b728b2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.951894 {25914ff5-f9dd-4ddc-9a55-175d7d8fef8b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.956453 {699ef1e1-f156-415d-8622-745b53062cbe} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.987658 {70d63626-1fbb-489c-b6ea-46a0567b17ad} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:19.993569 {722151c3-a9b0-4561-9e3a-1d48f29f6827} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.022976 {abcaa296-89d1-4bbd-9275-1fc453e86068} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.027557 {047ab3ed-352b-4fd1-8f73-3e37e25b050d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.057591 {6b815328-2da8-4fa3-97f6-73a21c3b01f1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.062077 {4a971198-7f58-4b99-8a4f-b2daff9f1c6d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.093527 {226f296a-14a5-4520-b98d-8d5d99140501} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.098204 {41e9d5a9-64bc-48fd-98f0-7e8ffef20278} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.130253 {7455a815-8033-447b-ae19-ecb1473d36c6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.134640 {8aa61cd8-368e-483b-b982-7051a7ba3837} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.164241 {81d57bd3-b317-4b8b-abc0-cd49f60ddc9d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.168740 {0c5cda5f-24df-4a42-8dbb-56bf25136c09} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.197977 {6daacab3-966e-4f4b-8542-b8d3418224be} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.202256 {f7f90e2c-754e-4177-9dab-3ff27d26e8be} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.238348 {90e90244-0c21-4dd2-9603-f89b2f93ff16} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.242716 {0218e165-2f06-4132-ac45-29a5e8b5d69e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.271237 {842d61b2-bd50-47cb-96cc-c89d627c4557} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.275480 {89c59a16-4dae-4cc0-800e-f429021fd569} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.304508 {e9d696b9-f85f-4e10-ae42-1cf49574a93a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.308857 {b358b7af-de26-42d1-965e-6747f20d5710} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.340686 {396f1a07-db15-481b-8f2d-5b7d4c55ec84} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.346514 {d5c89f7e-b0a9-435d-ad26-15825c40f5e5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.376483 {591e8be1-de5e-4b41-8516-125310d6b9f6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.381852 {e3e17728-9d29-44e1-a593-1d95a8e9cd19} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.417500 {ecb7eedf-6368-4060-adbb-fba20354f920} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.421741 {18fc2fa7-09ad-44d4-9cc3-50612274d94a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.451213 {17114c8a-9463-4c70-ac77-c6181520ce84} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.455464 {456a1868-fa09-4c07-8439-d12b7127b458} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.484071 {7ad7609b-d972-4ba0-a960-627c93a9675b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.488277 {39c90739-32e8-4079-932a-5768c230a0dc} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.517682 {95ddd3f3-5bcd-4265-8089-ae1fa9e7e57c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.521952 {3d34cd36-35a9-4ac1-a8ba-f09775867824} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.559705 {82746161-fb45-48d8-9095-2180a7cfb97b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.564957 {c0e15d93-4de6-4a79-9284-e26c451a2bc2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.610499 {9c89f274-e0be-4f98-ac78-1796725e0cd5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.616906 {c365e03c-4f7d-49b6-8ca6-05b31ad52b29} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.653925 {5a5dfa1f-452a-4371-9f16-24a40fe48b45} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.659995 {852f83f3-77b3-45cd-8e24-7c5ada6f725a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.695582 {c13fa2a1-7353-489b-b777-dd7c4a8910f0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.700837 {ae8aad39-824b-4dc3-ac2d-81890a584686} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.734103 {184bd1e5-3802-4805-8722-71dc244ff1e7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.738855 {37e36a10-6c38-4eba-bf62-302888a4949a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.743649 {0320f6a2-9055-4096-9a25-3ecc05537f57} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.748421 {c5e1651d-b24f-472a-9381-c040b15a8785} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.781298 {c58754eb-a630-4cbd-b478-7b6df6731c58} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.786145 {9e2ea8d0-8007-46fe-9708-d9fe722c5947} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.819800 {ed99c5cc-7eef-49a9-9a18-4ff5c44ffb86} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.824973 {37311630-ddae-4b94-9604-1d35f7640a27} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.857977 {3fdacdea-1206-49df-969c-935089fc405e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.863958 {252d64cb-fba8-4da8-a2c1-2dc2a3a57a71} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.896295 {ad9236ac-2a09-4080-87de-fd0694c88785} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.902358 {d71cf112-dd5a-4a53-96ef-c709137f33cf} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.934575 {9e53f86c-6822-4881-ad36-f7b6bd0c313d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.940409 {8e9a166f-2e44-417f-94d1-c3583d60f430} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.980619 {c9b7a5a0-d83e-4d2a-a23c-0b52b1e0acdd} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:20.985243 {5cd7aaea-1c9f-4952-bd06-37ff3ca5010c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.022250 {2a6a2714-898f-4f09-9568-050843a2dc17} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.026802 {79f88e5a-f0ca-44ab-8fac-12483d2a1ea6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.055087 {b519581a-8619-498e-b12d-35c2956a082f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.060776 {e64853c7-519e-4bde-91f2-5662658dd35c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.088767 {e7118161-2398-49e6-a9b7-8c028beefb09} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.093843 {16a7e16a-f3b3-4294-a2c9-49458e9397ce} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.129158 {416bfe21-aa0a-4b8d-af54-d7256a23b199} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.133101 {9700ec2e-1762-4fa0-823c-4e697d6d2f9f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.167084 {47cff90f-d2b3-46f6-89d9-f8026770bef3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.171544 {5d448d45-0d28-45a5-a105-6e639c1ffc6d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.206945 {3a8be4c7-08c3-45ed-a869-28388d209ed3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.211038 {cb1b6771-8fd5-456c-99ea-dea467509e07} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.239332 {c0f22c53-f251-40d0-922a-f4402204b8dc} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.243419 {cc587e7f-10bc-4a6d-94f0-75d36b341a64} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.272782 {f55a3397-ede3-4d80-9ef5-8f021ff2e1ea} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.277139 {1570d074-e72b-4d71-8843-70880f3dc693} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.312904 {bc878ba3-3864-4fe9-a161-344ea58de8f2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.316831 {9c6ade4b-6bf0-4a80-8cb3-52d5b411613c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.360880 {f3f0393d-ca93-465a-8f42-1462e89e2fc3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.366613 {67b99782-1899-4dfd-b6a2-fdc63f34e497} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.404496 {cc1153af-ef4d-4a28-bf39-a8e931ff94ee} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.408448 {08368686-064c-415a-ac9e-3478dcf99945} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.442690 {569ea309-6dc6-4e65-8034-963cffc86df9} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.447177 {f64c53eb-0f02-439c-9f4c-653a1395fa6d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.482489 {c1c3a832-f6bc-4b06-8f88-5860583961f2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.487032 {674931f1-be77-4d43-a1cf-079ab79def0f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.521585 {96ce195e-30bf-4a3d-8e7f-4771e738fd66} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.525858 {6ee02bca-2ff7-4f17-8e01-eb5e9ff40f46} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.555733 {f90fb44e-609a-4243-bdf1-ca1ba46e70ab} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.560186 {9310b49c-2e6a-4a76-945f-8439896d673b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.597428 {1a1bcaf8-771f-483a-a2a9-26c3b6389dd2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.601850 {7ea3d896-be69-438e-8f8b-c47399769250} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.635806 {459e76b3-a659-479d-a893-cdf0ec4c760c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.641221 {117beeaa-e4a7-40a5-828d-00ccb63100d3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.670832 {953fe791-17ad-41e2-94d4-6a4d21d6c316} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.675038 {67777efa-3636-48f0-a18c-f1bbb3f7cd5d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.709370 {5a48eb2a-782c-451d-b80e-5db52dae9547} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.713206 {e72f8682-acef-4ac1-b20d-c7fc80e2e829} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.716976 {48fd92e8-db30-4e3e-a339-07b1f413f3b1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.745014 {a8938533-738f-4d69-9024-31584f19a8c0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.749449 {2cb3fa80-9c75-4ee2-af6d-19fd1228deef} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.783188 {f6ec55fc-290e-4b08-8a9d-c701245c2336} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.787876 {547f4674-fd7a-4094-8cc3-05c31d77e0b2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.822522 {ae378b21-af6b-43a6-a312-94d280aa1d3e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.826206 {07e57cbf-325a-4c15-b0fb-4f69c5cbc2cf} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.859826 {c5027d5d-0c35-4d77-aff8-ec4ca9bcabae} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.863770 {383ee63a-47be-4745-bd37-04ffaabab9fe} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.892932 {6fae4e99-f838-40aa-97f6-213ae722a8ac} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.897416 {a9bda8c2-3d33-4d56-b0b2-0af63192f968} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.931079 {ba207198-33ff-465b-9db8-398e2002ea56} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.935494 {92d9d25f-9c4a-48b7-b129-771b64699e2e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.967052 {c442f81f-3674-4b8b-92f0-6be36bb3d916} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:21.970713 {b3953ddb-47d7-4f3f-96a5-25c582665063} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.003257 {57930ed8-fd61-4db2-9297-e075829f0a96} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.007453 {f05f9e22-8b6a-4ced-8037-09728cf5168d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.039587 {71b7c4ea-bac1-4306-9fee-128babaf2895} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.043201 {5e2e6745-be94-4486-a9df-31e217826d5c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.074663 {2318b65d-c534-4ef1-a6e1-14d53de61f50} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.078282 {dc2b480e-9e12-42e6-a335-df2563c77b7b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.081793 {e8a2aaba-ea2f-495a-8c42-59b0d1261e03} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.113369 {6cd7742e-e9bb-4578-a88b-b58f3cc00c45} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.116984 {409adf27-ba86-4aab-b0ce-e31bd00aaf6c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.147879 {4d8c7357-b35c-4e0a-b71c-13381db4bbfc} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.151655 {fd9aa6a8-67ce-438b-89dc-901c0e8974c1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.186383 {184ff725-f1b6-4ab1-ab07-d03a4b89c88e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.190097 {33d4e50c-a721-4e88-9828-1e03b2165ffd} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.224751 {c59dc5cc-3962-403d-92b6-90a5d2d0cf17} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.228239 {3625082e-9a39-47b4-8475-d1dea5d8a9f4} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.231705 {c110320b-7f40-4280-a831-743d58fd3ead} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.260927 {57a49126-2ae2-4d62-8d59-016ad27336f4} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.264474 {68e653d0-838f-4814-bf10-d1a65e24b8f6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.297853 {6cb0abe9-62a4-4add-b5ca-5d296b0e3479} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.301329 {1c60e3ee-f6a6-4376-8f15-aede2c9dbbd7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.304826 {4fec22c8-29ef-447b-9092-5ff8b68ef8cc} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.330141 {54afef42-8f7e-4ed5-917c-1dc7e090b415} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.333644 {0241f586-51cd-44e2-a8d6-5fa680edf7bd} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.337019 {6d860f1d-dc87-4f34-8f10-469228394ac9} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.363532 {3d667d64-aacc-4fdd-bacc-dbf61091a4b1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.367732 {7bfb07f4-654e-4e4a-b095-d1ecb2d38220} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.371268 {1d25d1ed-f480-4497-9c69-a335b3280106} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.404036 {07c1552a-7673-436d-8b6a-57a3fe7a2341} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.407538 {52cf0afb-37f0-4015-aed6-a7217aee7930} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.439175 {1b249ca3-7a79-475b-b7f1-73cb85e51166} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.442568 {b90152f7-7e61-4840-8b6e-34264cc7aa52} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.446038 {167d6b2e-f375-4f09-858c-cf38597299e5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.471179 {ecef70e6-9ac0-43a5-8d0e-97eb99c19954} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.474501 {810d0e2e-f3c2-4397-bd6b-ea45f793afe6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.477936 {70cdf0c2-1a75-42bc-ad18-da38edf34050} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.502297 {c5bc8a07-2df2-4460-85de-5ab900e08517} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.505521 {38ec563e-8b76-4538-84d9-0e6eeca66d4e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.508770 {5f2717be-df8f-4871-87af-b4ec38440007} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.540166 {2ab74fa3-e152-420c-9075-dc3b4aec2e3c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.543529 {af6e20fb-e2b0-4182-a2b3-b4839abc3ddf} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.547391 {4ef5ff91-0264-4acf-905b-bc3fb69243a3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.573467 {4dd8834c-fe40-48b4-9c7d-523995a0d675} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.577495 {58acdbc3-2e1e-4833-a824-32db2f9bd3f4} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.580934 {d78de5cd-99f3-4bfb-8fad-b71dd14f154d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.605796 {dc5b3f9c-b43f-453c-a037-e7bb9d19ee9a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.609052 {b7ae5542-dba6-43ef-abab-e00399059e5c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.612499 {be8ede28-54df-468c-b612-d91f30c17807} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.642955 {083740ab-aa80-4f94-8023-eb73472d72c5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.646944 {eb621a03-4dae-4e0f-baf1-29ce98c8ba7e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.650230 {f0299548-6254-4d83-9d24-b5c11f8b7421} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.676136 {1fb4a4df-0bd8-4447-b247-866b9daace51} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.679469 {1d60ef94-8041-4b80-a734-2fccf4c06695} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.682835 {86f87c4d-00fe-4a3c-b69e-81e25b8bda72} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.708673 {9f33e957-7c5d-4ace-bda4-866e755d5fee} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.712626 {587b3b9b-bc71-45c5-b5c6-87f32b4944e5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.716039 {980deada-45ec-4017-a8b4-dcf275ac3dc7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.741334 {7472d9e5-ec6b-4f5b-a43d-86514852035c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.745204 {2196a606-cd48-44fc-9fa1-afe1eb322819} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.748456 {7b31c11e-f3bd-422a-b0b7-4e7e29b62f2e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.779923 {99a855ec-85f3-42b6-8673-1d6c37f604c6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.783812 {d40b0c66-8548-446c-a6f2-e8f8ade943de} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.787111 {437eceb5-2d4e-4c86-928d-d24d7f1aa863} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.815557 {e78f004c-8d46-4547-90dc-a4300dc737be} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.818804 {fc5be34e-be00-4bbd-986b-8e7cb65a4ec3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.822552 {4512f33d-4833-441f-97a9-8b254d4bf3e2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.852604 {91859ba8-d9c8-4d8f-839c-9ff115248b3b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.855874 {bb5a0059-fb21-43dc-95f4-e584cdae393c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.859080 {7163fd83-d268-4598-9518-b66a7df1bbae} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.886878 {b83e3cf7-3ea3-4004-bd6c-80ab09e2a83a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.891245 {97bf6d42-10ed-415b-b06d-8fb08101e891} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.927016 {953f28cb-d0d0-4d5e-a873-15e70bbcb537} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.930995 {792409bb-5d8f-489b-b31c-acdb1f16ea74} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.934174 {f6d79436-9640-4f52-abfc-5194942623a4} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.968763 {828bc1d4-7a36-4d2e-965d-acdeac867afa} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.972760 {d914976f-6de3-4721-8f11-aa73175e3dab} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:22.976011 {45f0bc16-7743-40ad-912d-3aaac7b7078a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:23.008075 {9c1c97e8-d125-4e8f-84c8-bc722d100a5d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:23.012461 {73424367-e590-4d5e-8305-99e188abf71a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:23.015673 {91652558-16fc-4891-a45b-9a7310a942bc} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:23.042437 {c9a9f69d-c3d2-4291-8f91-988e5ae3a341} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:23.046318 {7fd4a8cd-2bf4-4c36-8ed2-001c4412c1cb} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:23.050564 {851144f0-0da6-4a32-8e34-e46e530b3620} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:23.090745 {a0971454-8b1f-4971-9afa-5aa942caffd7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + +[milovidov-Pro-P30] 2018.11.28 22:08:23.095710 {fd284264-56e0-47ee-b66c-623eae90caee} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: + +0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] +1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] +2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] +3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] +4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] +5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] +6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] +7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] +8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] +9. ./clickhouse-server() [0xee65008] +10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] +11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] +12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] +13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] +14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] +15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] +16. ./clickhouse-server() [0x10df8959] +17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] +18. ./clickhouse-server() [0x10dfa3d4] +19. ./clickhouse-server() [0x10dfa35d] +20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] +21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] +22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] + diff --git a/dbms/tests/queries/0_stateless/00763_lock_buffer.sh b/dbms/tests/queries/0_stateless/00763_lock_buffer.sh new file mode 100755 index 00000000000..582eee51345 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00763_lock_buffer.sh @@ -0,0 +1,29 @@ +#!/usr/bin/env bash +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.mt" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.buffer" + +${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.buffer (s String) ENGINE = Buffer(test, mt, 1, 1, 1, 1, 1, 1, 1)" + + +function thread1() +{ + seq 1 500 | sed -r -e 's/.+/DROP TABLE IF EXISTS test.mt; CREATE TABLE test.mt (s String) ENGINE = MergeTree ORDER BY s; INSERT INTO test.mt SELECT toString(number) FROM numbers(10);/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||: +} + +function thread2() +{ + seq 1 1000 | sed -r -e 's/.+/SELECT count() FROM test.buffer;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60' +} + +thread1 & +thread2 & + +wait + +${CLICKHOUSE_CLIENT} --query="DROP TABLE test.mt" +${CLICKHOUSE_CLIENT} --query="DROP TABLE test.buffer" From b92bcb3385c1e375a8a7204048a4c9ab96a24b1c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Nov 2018 22:12:19 +0300 Subject: [PATCH 40/47] Added test #3694 --- .../0_stateless/00763_lock_buffer.reference | 10374 ---------------- .../queries/0_stateless/00763_lock_buffer.sh | 2 +- 2 files changed, 1 insertion(+), 10375 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00763_lock_buffer.reference b/dbms/tests/queries/0_stateless/00763_lock_buffer.reference index effae2417a9..e69de29bb2d 100644 --- a/dbms/tests/queries/0_stateless/00763_lock_buffer.reference +++ b/dbms/tests/queries/0_stateless/00763_lock_buffer.reference @@ -1,10374 +0,0 @@ -[milovidov-Pro-P30] 2018.11.28 22:08:15.804398 {b382b34d-e62b-4a0a-bb8b-5671b9d82a40} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:15.839436 {12df6433-eab4-4494-8647-1eee0dda68a3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:15.879572 {8117c51b-d65b-4ec1-b4b7-e9a77407ced2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:15.886553 {aa40f63b-8632-45a2-bc38-ef891efb1dff} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:15.924566 {f20c3ceb-5efb-4d69-96a0-d8eadf558a77} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:15.930785 {06b4ed5e-9851-4ce5-b545-04af59655729} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:15.967409 {a0a04a9d-5088-47e9-a3de-944a28746862} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:15.973558 {4b580694-b7a4-422e-bb3a-4b185ddf8ae8} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.010300 {9dbfaeff-08e5-47ed-8bbd-d9276446382c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.016618 {c994f4e5-ac82-49d6-90d2-a9c857a7fe61} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.052850 {1d26d531-2005-4f49-9f66-6962a3ebf796} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.059333 {eb95a219-c505-436b-93b9-ebbaf36ecbb0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.097461 {6276b2af-ce73-4ac4-826a-a73dc1a3eaf3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.105200 {0caa94fe-74b6-4471-b9ac-c7d85fe33007} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.143043 {93b81cb5-525b-4e94-86c0-d5919d7608ea} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.150518 {fd8c908a-e0b5-4fb2-b419-bd08c21c3c13} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.190703 {7f788212-7a46-4f3c-acb4-c583b28e9df2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.197392 {878132f9-5219-4aea-914b-0dbd7b58a764} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.233792 {8d04aba7-9125-467e-8ea6-0e16bc7bf7c6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.239797 {094bd4d8-0eac-4e84-8b3c-2ea07d3b4916} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.278886 {d599a104-eaeb-4b60-b603-12b65892dc38} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.285176 {69888f58-b324-43f8-80c5-b0b8e6561649} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.324470 {583d7fc9-db34-4550-81ae-6ff31a1d96b8} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.330644 {b518f495-9f63-4b92-aa39-75a24398866b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.369672 {28378d76-b475-49ee-8908-9be52469e0a1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.375686 {b0f79188-0e4f-4980-9d1f-78ce9db2f7cf} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.412233 {6fc614df-02d1-4c3a-aff1-f03bc3ef0aaa} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.418180 {54059a5c-7754-4552-b140-c0e4a86ef40e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.458263 {5fa1e884-db86-451a-80a0-d8274019c10c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.465598 {b5bec235-c220-4a4b-b5e4-2c57179c2e91} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.502333 {78149d68-ab22-4898-9596-15b207f99ef6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.509010 {6c533a36-c9fc-4c79-b334-9fdb066d9464} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.547343 {8f4e117f-c957-4955-a7a3-466a3df25a89} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.553354 {acada29b-8541-4054-bb97-bb35e88eab27} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.590795 {89266bb1-4e35-44ec-bf2d-9bc1c26f1a74} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.596932 {7b0f1862-2909-4b1a-b476-e169d65b5dfc} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.634903 {f3449d01-002e-46a5-9bd4-120777f6bfec} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.640730 {fe17c38b-0b23-4615-814f-7903fb2cde0c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.678091 {8c0bcdbb-ca41-4827-b170-cd933fb8b651} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.683945 {872c146c-d6bd-4cf1-b61d-0e26884097d1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.719134 {931a96e6-7973-403c-9ba9-951a22711f14} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.725387 {56c9ef93-b51e-49e1-9044-75a8e1adb420} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.761859 {4ab5c13c-b88f-47f0-b95e-0fe36678b000} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.769417 {585d204d-88ea-43f2-bbe3-31caa4626e4b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.805721 {692534b6-6558-4913-92e1-8aaefe56ed57} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.811425 {e027f56f-f8a1-4b24-8d49-6f4bd5b340ed} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.846068 {610c8387-3df4-424f-9c5e-9d7eb18debae} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.851569 {76a2eb5e-1370-4e12-8715-21750af16bed} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.887480 {86c45297-16ab-473d-b201-e2236e5b6116} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.893490 {9f2e0e03-8363-48b8-8829-d922084ba301} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.929084 {70ef1316-e73c-4ac8-b417-e53dd38722ea} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.936277 {5d2fa797-629c-4568-99b6-0e65ab27be7f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.972299 {f3a0a47b-2399-4022-9d54-f37af9eff412} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:16.978024 {5a74a293-7e13-41b9-a80d-f5a4e9b2e868} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.016729 {d92291d9-f4fc-4823-bbc6-71785fb5d07c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.022342 {30486679-6c7f-4877-88b3-f3242d483466} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.060484 {70852fb6-c64c-47e4-b1f4-e8d7f8432522} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.066117 {2fba7d1c-b08c-4a2b-bea4-9d4dd80e4975} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.102132 {c3918e15-7842-4887-bde0-1928a36bda2c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.107582 {97a724d7-97d7-4466-af61-23e3e34ba601} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.142054 {771188c8-f84f-4463-a559-1583432eba4b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.147795 {b09b2a35-a72a-4d3e-8d74-f09f57d449db} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.184394 {f982786e-131d-42e5-8214-ae3aab11328d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.217858 {38f72a08-ecd5-45ab-8899-ff2fbcec0579} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.223712 {742b544a-7c8d-4466-bf15-f9584991e311} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.257521 {5382a35e-3be7-4a30-9b7b-058a8601f264} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.262962 {4bfae50b-e211-4be2-9d76-d0aba72eb2eb} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.301359 {8c2b0b6f-0f3b-4c29-83db-6840b05f2837} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.307158 {a6dc969f-192c-46c2-ab0e-00856a63bcb3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.343061 {a129bb03-266b-4562-8a6d-00a0e73ee8f1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.350189 {22e13b6d-958b-4429-980a-9637f0d2f676} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.384017 {ae678170-bcc3-4679-b883-9715af969b78} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.389498 {f82bd3f7-1b76-44bc-af73-7dbaf1c699f0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.425316 {80c8d353-e8f9-423c-8a24-c3d374bb3145} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.431225 {df4e4518-3853-4528-b159-ecd9eda8ff09} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.465149 {5b535596-a956-43b1-a2f7-2da7c656b5f6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.470614 {8f63bd16-446a-4fae-8ad7-f7dee705a21a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.506173 {b833c309-001b-4a29-a16d-9486c6f9cd20} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.511528 {0268a36b-c2ab-4dbc-8382-23964f1259b1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.544800 {987ed9ac-be0a-4ef6-9171-0f50740ad8db} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.550231 {a3f8b19b-a42e-4e5b-a0f3-914421f11405} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.583151 {75233a53-9450-4cfb-9a2a-791ba27cd2d7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.588621 {579b7452-06f6-400a-b25b-201d81e8f705} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.626467 {99852f0e-0063-499a-8391-4f8b63a1d455} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.632010 {e8743ea8-ab0e-4f89-bff5-e07ae11c14d1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.667868 {6240085d-c1fb-4173-b9ea-a8435fe6a445} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.674577 {511fd4e2-d615-4413-a560-889ac496b886} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.709817 {dc2485cd-6028-4cbc-bf9c-7bc17393260e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.715129 {8c7fba23-6e43-4f9b-ad3e-3d2116eb1e08} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.748908 {e404b337-cd8a-4a35-8d56-750dd327d2be} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.754181 {6291c4ae-3d6e-496d-87f5-eed780a6a2b0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.788946 {8326522b-1a7d-45ae-ad79-97bf5f44c617} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.794666 {02a7b413-613b-42d8-ab7c-4342f2b6a669} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.829819 {40180165-0671-415a-b3ee-75711f5bcf3b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.835307 {08e18b86-0925-40ac-a73c-739d3758ffc1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.871530 {bb1b2073-4709-40ca-aed8-1eb0d5983cab} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.878301 {3630c665-036c-4b70-b9f0-3d99e79e983d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.913240 {96fda9b7-54f2-4b16-84b8-f551a5ec8abf} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.918661 {555cec54-7d2d-481e-994e-6ec08626cefd} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.952820 {6e9bb6ac-4d91-4fcc-b3cd-f0fc440f50d1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.958480 {85942327-3b97-475b-9b94-07fdea68e5fa} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.993680 {80aee0e2-7712-4c5e-a919-d9f0f64ca0da} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:17.999245 {cd705bb9-9402-4d61-b05a-d8142209b78d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.033388 {abaf0831-9199-4cb8-8010-8637caf73715} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.038605 {dfd8a3a0-5e50-4afe-ae4b-d1a2b88383f7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.073803 {b3737f44-61a2-44b1-9dae-3fff37021d32} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.078988 {573e82f0-3fa6-4ed5-a6bd-9836d4c376d2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.116072 {8005c879-14fc-4f4d-a65b-cdd8ccb95f02} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.121410 {04cc6056-7e88-425f-b5c0-ccd3f913b128} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.157133 {c0475c4c-86b2-4f77-90ea-e787beff8e79} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.162408 {88849363-2290-4e35-9bfc-4fa059991725} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.195365 {6dacc238-b4e6-4229-a124-5bdce40b2315} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.202200 {ecded22b-1ffa-4a09-9f98-65b60daed8f2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.239109 {85a22162-577b-436b-8ea5-5dd04b971362} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.244307 {1c5fbc39-87fc-4d1c-b0e9-f151f6e5320d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.277030 {d88bc3fa-0534-40f9-bd34-7f102702b971} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.282619 {fe704672-05a9-431b-9248-27ba2d50da2b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.316182 {fe0bcc92-42f2-471c-a54a-99bee226c12c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.321458 {d2a4cb2d-d450-419f-86eb-fc34543ad4e2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.354352 {a12c0c42-2a78-48ed-bfd6-9b0c8486b97f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.359477 {f73fd3f6-ada8-4d16-94b7-d682753e8814} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.392610 {e18daa32-8309-46ca-9b21-89d922f52df9} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.397918 {f518e531-736a-4811-855d-950ba52acd3d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.431508 {fbf7a0c3-ce49-4b92-8c0f-f68049d04242} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.438271 {f33025eb-42be-4d3c-b7a2-02f47668e987} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.471301 {88aae2db-b56f-4b8b-9545-123c5a22052a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.476369 {6739091d-fdd5-4a5b-ba7f-16b468222467} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.510199 {ff128f44-605b-49a2-863c-d45c557ab6ef} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.516160 {d1d3b6d0-7e38-473c-8455-e16fb059baae} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.549400 {142da820-42d7-4d60-8f90-a014b8d5978f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.554544 {d0bf63f6-9458-40fd-96e7-8430634896e7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.587420 {b7df241e-b337-4a9c-a400-694ece7745ac} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.592825 {d62c459f-a84a-406d-96dc-50615862df94} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.624438 {b1dd3f0a-d2c2-420a-89fe-739caffa972c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.629489 {a88a9512-0d41-4b74-b010-d316d099fc6c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.660508 {79e5a325-c92a-41db-9743-eba05c45bfa1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.665771 {f1cdea6e-d097-4633-a4dc-04542c1f9e01} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.699952 {94144751-cb9c-449b-8090-9cd064d4e10c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.704897 {85ae60e6-fd10-4abd-b42f-a29208b46b3b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.735563 {2a6c79f1-3839-46c0-b310-2850b7e23eba} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.740511 {a3aedcd6-49a7-4309-a191-ca698e6a419e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.771576 {b0932dd8-dce1-4339-9b7d-b8340b1874a0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.776862 {0e71cf19-2563-48ed-bf66-f15a309f29f5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.810621 {75dc62a3-8c9f-44e9-b194-233f7819bda8} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.815959 {eee05c7a-05ed-47e6-9032-7ae4cc27c262} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.851936 {f3391943-b4db-4fd2-bcc5-67c522672a2b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.857406 {1d40ead8-ed6f-4552-9e06-9889cb616e06} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.889979 {5b02c382-5c76-4fc4-b52f-a367dd2a34e2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.894979 {a4102dba-66e7-4c23-a79f-18372bc18eda} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.928024 {74a886a1-18a0-4da7-8179-b305e14306b4} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.932999 {89679622-084d-422a-a412-8da65ab4a152} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.967572 {c4d1efeb-abb5-4afc-bce9-f0a83ddc52f3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:18.972741 {ecfef9e5-7656-4440-93d7-6b0de9dc5936} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.005420 {16b6a55c-3f1a-40fe-9197-8ef1fee26ad6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.010876 {4bf80c2f-f584-4634-a4cb-afd2da5f17f2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.042372 {3f8516f7-1630-413b-9f1a-c6e10a3489d3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.047239 {416eff96-a99c-44d1-8a9c-43aea8620633} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.052187 {c8f87f7a-86dc-487d-a2e7-0d345426fba0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.084790 {4a5ea3c0-1d66-488f-9735-10e35c4fa20c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.091202 {ec69c65b-8666-4d5d-a9e0-cc5f1f9bc5d7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.124920 {6e993fe5-d826-4c93-9afb-cf56666d99de} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.129856 {bfc0f2b4-ed16-471b-94d7-a84c1e712700} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.163197 {934d9b62-9279-4e1a-ba5e-dc49b1537c41} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.167999 {ecc36b79-a499-43ed-8d11-d9246353a6a7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.199826 {41623caa-87c5-4c45-90e5-786eb64927d8} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.204546 {a3a8403a-1db4-4b1a-bcda-b3e6df050b7d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.235716 {58068b04-dd39-4efc-90c3-472a80c63581} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.242427 {08fe6947-b10c-4d2c-8f66-874867f96e26} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.281260 {998ed272-ce7f-4d73-8e8e-7337b83411e3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.286139 {0539dbe3-2fa1-42ec-ac7d-a6fc178e08e6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.318034 {30d2f17f-eb27-445d-aa94-4d1849cd17d4} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.324364 {14205a4d-d99c-4f36-be39-b92fffd1f4d1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.356909 {0b482022-ef79-49e9-a280-f7654a59b652} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.362271 {f483b92e-e257-4569-9908-be72d6467878} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.395056 {8690f680-f715-47be-8f60-bfc23073ccff} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.401412 {b00ecf03-4094-4d84-a999-e16bd3c955e6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.439565 {aaf84738-51d9-4664-b0d1-05eab0673069} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.444273 {f5942cd3-9ac8-44f0-bffa-649635c7f846} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.478090 {aed97b0a-37a1-4858-bbfa-431a5e8f7a8a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.483467 {18de9ba6-96a6-44af-9342-1841e273ad1b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.515387 {73491b80-9470-464e-ae2a-37dad11887a5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.520106 {b56c5c49-5af0-4329-b81f-cf0ee0a1807f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.550740 {c9fac8fd-b266-4612-a231-4e8a9dfc7c09} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.555584 {18f6af96-bcc5-4877-ba87-994492302d83} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.588382 {680e7447-65f1-4a34-b326-18d7eef0ac4d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.593073 {1c0d3ac2-de89-4fe6-94f5-1c31f68e9ae5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.625452 {0d8c5081-3a3b-4c0c-842c-04dbcd46796d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.631467 {943503e7-4311-4ecf-85d6-09103b6301b9} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.662721 {92b83212-21fb-4562-b152-3d83c1984ec2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.667347 {e9313bf0-74ba-419f-b388-f9f74b916d34} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.697209 {c1ae5516-ce7a-41bc-bc3a-4ed84cbeac31} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.701821 {169682d3-59ee-4315-a69d-d30a63484a2e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.732723 {dc58c84f-e2a7-487e-8635-f7b90e37e6b0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.737300 {abb290a9-2475-4f3e-92e7-210f422e6c35} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.768092 {022060f1-46a9-4c06-ab85-eb89228d5ac1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.772777 {c54a77a3-6bd3-4a1e-a463-8c1617fe1729} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.806728 {622acf90-b971-469e-a275-efaeb127d5fe} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.811228 {3ceaed60-d474-402f-9475-7656e201e555} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.843577 {be73cadf-ddec-4986-a225-bc3bfafd746e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.848357 {665df9d2-5875-48a7-a342-aff8c1ecd46c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.880431 {970e148e-5956-4b87-bce7-be4fbfdf1d9f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.885736 {27463163-3b99-4724-b6eb-e4273078d8a9} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.916999 {c1fa3ba7-9304-41f7-975c-907ddd2ce75d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.921450 {fad8c602-fad8-4247-9454-b36481b728b2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.951894 {25914ff5-f9dd-4ddc-9a55-175d7d8fef8b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.956453 {699ef1e1-f156-415d-8622-745b53062cbe} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.987658 {70d63626-1fbb-489c-b6ea-46a0567b17ad} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:19.993569 {722151c3-a9b0-4561-9e3a-1d48f29f6827} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.022976 {abcaa296-89d1-4bbd-9275-1fc453e86068} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.027557 {047ab3ed-352b-4fd1-8f73-3e37e25b050d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.057591 {6b815328-2da8-4fa3-97f6-73a21c3b01f1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.062077 {4a971198-7f58-4b99-8a4f-b2daff9f1c6d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.093527 {226f296a-14a5-4520-b98d-8d5d99140501} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.098204 {41e9d5a9-64bc-48fd-98f0-7e8ffef20278} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.130253 {7455a815-8033-447b-ae19-ecb1473d36c6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.134640 {8aa61cd8-368e-483b-b982-7051a7ba3837} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.164241 {81d57bd3-b317-4b8b-abc0-cd49f60ddc9d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.168740 {0c5cda5f-24df-4a42-8dbb-56bf25136c09} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.197977 {6daacab3-966e-4f4b-8542-b8d3418224be} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.202256 {f7f90e2c-754e-4177-9dab-3ff27d26e8be} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.238348 {90e90244-0c21-4dd2-9603-f89b2f93ff16} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.242716 {0218e165-2f06-4132-ac45-29a5e8b5d69e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.271237 {842d61b2-bd50-47cb-96cc-c89d627c4557} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.275480 {89c59a16-4dae-4cc0-800e-f429021fd569} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.304508 {e9d696b9-f85f-4e10-ae42-1cf49574a93a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.308857 {b358b7af-de26-42d1-965e-6747f20d5710} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.340686 {396f1a07-db15-481b-8f2d-5b7d4c55ec84} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.346514 {d5c89f7e-b0a9-435d-ad26-15825c40f5e5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.376483 {591e8be1-de5e-4b41-8516-125310d6b9f6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.381852 {e3e17728-9d29-44e1-a593-1d95a8e9cd19} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.417500 {ecb7eedf-6368-4060-adbb-fba20354f920} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.421741 {18fc2fa7-09ad-44d4-9cc3-50612274d94a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.451213 {17114c8a-9463-4c70-ac77-c6181520ce84} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.455464 {456a1868-fa09-4c07-8439-d12b7127b458} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.484071 {7ad7609b-d972-4ba0-a960-627c93a9675b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.488277 {39c90739-32e8-4079-932a-5768c230a0dc} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.517682 {95ddd3f3-5bcd-4265-8089-ae1fa9e7e57c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.521952 {3d34cd36-35a9-4ac1-a8ba-f09775867824} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.559705 {82746161-fb45-48d8-9095-2180a7cfb97b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.564957 {c0e15d93-4de6-4a79-9284-e26c451a2bc2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.610499 {9c89f274-e0be-4f98-ac78-1796725e0cd5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.616906 {c365e03c-4f7d-49b6-8ca6-05b31ad52b29} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.653925 {5a5dfa1f-452a-4371-9f16-24a40fe48b45} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.659995 {852f83f3-77b3-45cd-8e24-7c5ada6f725a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.695582 {c13fa2a1-7353-489b-b777-dd7c4a8910f0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.700837 {ae8aad39-824b-4dc3-ac2d-81890a584686} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.734103 {184bd1e5-3802-4805-8722-71dc244ff1e7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.738855 {37e36a10-6c38-4eba-bf62-302888a4949a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.743649 {0320f6a2-9055-4096-9a25-3ecc05537f57} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.748421 {c5e1651d-b24f-472a-9381-c040b15a8785} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.781298 {c58754eb-a630-4cbd-b478-7b6df6731c58} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.786145 {9e2ea8d0-8007-46fe-9708-d9fe722c5947} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.819800 {ed99c5cc-7eef-49a9-9a18-4ff5c44ffb86} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.824973 {37311630-ddae-4b94-9604-1d35f7640a27} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.857977 {3fdacdea-1206-49df-969c-935089fc405e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.863958 {252d64cb-fba8-4da8-a2c1-2dc2a3a57a71} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.896295 {ad9236ac-2a09-4080-87de-fd0694c88785} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.902358 {d71cf112-dd5a-4a53-96ef-c709137f33cf} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.934575 {9e53f86c-6822-4881-ad36-f7b6bd0c313d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.940409 {8e9a166f-2e44-417f-94d1-c3583d60f430} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.980619 {c9b7a5a0-d83e-4d2a-a23c-0b52b1e0acdd} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:20.985243 {5cd7aaea-1c9f-4952-bd06-37ff3ca5010c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.022250 {2a6a2714-898f-4f09-9568-050843a2dc17} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.026802 {79f88e5a-f0ca-44ab-8fac-12483d2a1ea6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.055087 {b519581a-8619-498e-b12d-35c2956a082f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.060776 {e64853c7-519e-4bde-91f2-5662658dd35c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.088767 {e7118161-2398-49e6-a9b7-8c028beefb09} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.093843 {16a7e16a-f3b3-4294-a2c9-49458e9397ce} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.129158 {416bfe21-aa0a-4b8d-af54-d7256a23b199} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.133101 {9700ec2e-1762-4fa0-823c-4e697d6d2f9f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.167084 {47cff90f-d2b3-46f6-89d9-f8026770bef3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.171544 {5d448d45-0d28-45a5-a105-6e639c1ffc6d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.206945 {3a8be4c7-08c3-45ed-a869-28388d209ed3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.211038 {cb1b6771-8fd5-456c-99ea-dea467509e07} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.239332 {c0f22c53-f251-40d0-922a-f4402204b8dc} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.243419 {cc587e7f-10bc-4a6d-94f0-75d36b341a64} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.272782 {f55a3397-ede3-4d80-9ef5-8f021ff2e1ea} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.277139 {1570d074-e72b-4d71-8843-70880f3dc693} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.312904 {bc878ba3-3864-4fe9-a161-344ea58de8f2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.316831 {9c6ade4b-6bf0-4a80-8cb3-52d5b411613c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.360880 {f3f0393d-ca93-465a-8f42-1462e89e2fc3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.366613 {67b99782-1899-4dfd-b6a2-fdc63f34e497} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.404496 {cc1153af-ef4d-4a28-bf39-a8e931ff94ee} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.408448 {08368686-064c-415a-ac9e-3478dcf99945} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.442690 {569ea309-6dc6-4e65-8034-963cffc86df9} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.447177 {f64c53eb-0f02-439c-9f4c-653a1395fa6d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.482489 {c1c3a832-f6bc-4b06-8f88-5860583961f2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.487032 {674931f1-be77-4d43-a1cf-079ab79def0f} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.521585 {96ce195e-30bf-4a3d-8e7f-4771e738fd66} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.525858 {6ee02bca-2ff7-4f17-8e01-eb5e9ff40f46} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.555733 {f90fb44e-609a-4243-bdf1-ca1ba46e70ab} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.560186 {9310b49c-2e6a-4a76-945f-8439896d673b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.597428 {1a1bcaf8-771f-483a-a2a9-26c3b6389dd2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.601850 {7ea3d896-be69-438e-8f8b-c47399769250} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.635806 {459e76b3-a659-479d-a893-cdf0ec4c760c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.641221 {117beeaa-e4a7-40a5-828d-00ccb63100d3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.670832 {953fe791-17ad-41e2-94d4-6a4d21d6c316} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.675038 {67777efa-3636-48f0-a18c-f1bbb3f7cd5d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.709370 {5a48eb2a-782c-451d-b80e-5db52dae9547} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.713206 {e72f8682-acef-4ac1-b20d-c7fc80e2e829} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.716976 {48fd92e8-db30-4e3e-a339-07b1f413f3b1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.745014 {a8938533-738f-4d69-9024-31584f19a8c0} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.749449 {2cb3fa80-9c75-4ee2-af6d-19fd1228deef} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.783188 {f6ec55fc-290e-4b08-8a9d-c701245c2336} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.787876 {547f4674-fd7a-4094-8cc3-05c31d77e0b2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.822522 {ae378b21-af6b-43a6-a312-94d280aa1d3e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.826206 {07e57cbf-325a-4c15-b0fb-4f69c5cbc2cf} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.859826 {c5027d5d-0c35-4d77-aff8-ec4ca9bcabae} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.863770 {383ee63a-47be-4745-bd37-04ffaabab9fe} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.892932 {6fae4e99-f838-40aa-97f6-213ae722a8ac} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.897416 {a9bda8c2-3d33-4d56-b0b2-0af63192f968} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.931079 {ba207198-33ff-465b-9db8-398e2002ea56} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.935494 {92d9d25f-9c4a-48b7-b129-771b64699e2e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.967052 {c442f81f-3674-4b8b-92f0-6be36bb3d916} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:21.970713 {b3953ddb-47d7-4f3f-96a5-25c582665063} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.003257 {57930ed8-fd61-4db2-9297-e075829f0a96} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.007453 {f05f9e22-8b6a-4ced-8037-09728cf5168d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.039587 {71b7c4ea-bac1-4306-9fee-128babaf2895} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.043201 {5e2e6745-be94-4486-a9df-31e217826d5c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.074663 {2318b65d-c534-4ef1-a6e1-14d53de61f50} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.078282 {dc2b480e-9e12-42e6-a335-df2563c77b7b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.081793 {e8a2aaba-ea2f-495a-8c42-59b0d1261e03} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.113369 {6cd7742e-e9bb-4578-a88b-b58f3cc00c45} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.116984 {409adf27-ba86-4aab-b0ce-e31bd00aaf6c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.147879 {4d8c7357-b35c-4e0a-b71c-13381db4bbfc} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.151655 {fd9aa6a8-67ce-438b-89dc-901c0e8974c1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.186383 {184ff725-f1b6-4ab1-ab07-d03a4b89c88e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.190097 {33d4e50c-a721-4e88-9828-1e03b2165ffd} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.224751 {c59dc5cc-3962-403d-92b6-90a5d2d0cf17} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.228239 {3625082e-9a39-47b4-8475-d1dea5d8a9f4} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.231705 {c110320b-7f40-4280-a831-743d58fd3ead} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.260927 {57a49126-2ae2-4d62-8d59-016ad27336f4} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.264474 {68e653d0-838f-4814-bf10-d1a65e24b8f6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.297853 {6cb0abe9-62a4-4add-b5ca-5d296b0e3479} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.301329 {1c60e3ee-f6a6-4376-8f15-aede2c9dbbd7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.304826 {4fec22c8-29ef-447b-9092-5ff8b68ef8cc} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.330141 {54afef42-8f7e-4ed5-917c-1dc7e090b415} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.333644 {0241f586-51cd-44e2-a8d6-5fa680edf7bd} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.337019 {6d860f1d-dc87-4f34-8f10-469228394ac9} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.363532 {3d667d64-aacc-4fdd-bacc-dbf61091a4b1} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.367732 {7bfb07f4-654e-4e4a-b095-d1ecb2d38220} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.371268 {1d25d1ed-f480-4497-9c69-a335b3280106} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.404036 {07c1552a-7673-436d-8b6a-57a3fe7a2341} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.407538 {52cf0afb-37f0-4015-aed6-a7217aee7930} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.439175 {1b249ca3-7a79-475b-b7f1-73cb85e51166} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.442568 {b90152f7-7e61-4840-8b6e-34264cc7aa52} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.446038 {167d6b2e-f375-4f09-858c-cf38597299e5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.471179 {ecef70e6-9ac0-43a5-8d0e-97eb99c19954} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.474501 {810d0e2e-f3c2-4397-bd6b-ea45f793afe6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.477936 {70cdf0c2-1a75-42bc-ad18-da38edf34050} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.502297 {c5bc8a07-2df2-4460-85de-5ab900e08517} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.505521 {38ec563e-8b76-4538-84d9-0e6eeca66d4e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.508770 {5f2717be-df8f-4871-87af-b4ec38440007} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.540166 {2ab74fa3-e152-420c-9075-dc3b4aec2e3c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.543529 {af6e20fb-e2b0-4182-a2b3-b4839abc3ddf} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.547391 {4ef5ff91-0264-4acf-905b-bc3fb69243a3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.573467 {4dd8834c-fe40-48b4-9c7d-523995a0d675} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.577495 {58acdbc3-2e1e-4833-a824-32db2f9bd3f4} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.580934 {d78de5cd-99f3-4bfb-8fad-b71dd14f154d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.605796 {dc5b3f9c-b43f-453c-a037-e7bb9d19ee9a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.609052 {b7ae5542-dba6-43ef-abab-e00399059e5c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.612499 {be8ede28-54df-468c-b612-d91f30c17807} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.642955 {083740ab-aa80-4f94-8023-eb73472d72c5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.646944 {eb621a03-4dae-4e0f-baf1-29ce98c8ba7e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.650230 {f0299548-6254-4d83-9d24-b5c11f8b7421} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.676136 {1fb4a4df-0bd8-4447-b247-866b9daace51} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.679469 {1d60ef94-8041-4b80-a734-2fccf4c06695} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.682835 {86f87c4d-00fe-4a3c-b69e-81e25b8bda72} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.708673 {9f33e957-7c5d-4ace-bda4-866e755d5fee} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.712626 {587b3b9b-bc71-45c5-b5c6-87f32b4944e5} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.716039 {980deada-45ec-4017-a8b4-dcf275ac3dc7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.741334 {7472d9e5-ec6b-4f5b-a43d-86514852035c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.745204 {2196a606-cd48-44fc-9fa1-afe1eb322819} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.748456 {7b31c11e-f3bd-422a-b0b7-4e7e29b62f2e} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.779923 {99a855ec-85f3-42b6-8673-1d6c37f604c6} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.783812 {d40b0c66-8548-446c-a6f2-e8f8ade943de} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.787111 {437eceb5-2d4e-4c86-928d-d24d7f1aa863} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.815557 {e78f004c-8d46-4547-90dc-a4300dc737be} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.818804 {fc5be34e-be00-4bbd-986b-8e7cb65a4ec3} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.822552 {4512f33d-4833-441f-97a9-8b254d4bf3e2} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.852604 {91859ba8-d9c8-4d8f-839c-9ff115248b3b} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.855874 {bb5a0059-fb21-43dc-95f4-e584cdae393c} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.859080 {7163fd83-d268-4598-9518-b66a7df1bbae} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.886878 {b83e3cf7-3ea3-4004-bd6c-80ab09e2a83a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.891245 {97bf6d42-10ed-415b-b06d-8fb08101e891} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.927016 {953f28cb-d0d0-4d5e-a873-15e70bbcb537} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.930995 {792409bb-5d8f-489b-b31c-acdb1f16ea74} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.934174 {f6d79436-9640-4f52-abfc-5194942623a4} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.968763 {828bc1d4-7a36-4d2e-965d-acdeac867afa} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.972760 {d914976f-6de3-4721-8f11-aa73175e3dab} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:22.976011 {45f0bc16-7743-40ad-912d-3aaac7b7078a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:23.008075 {9c1c97e8-d125-4e8f-84c8-bc722d100a5d} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:23.012461 {73424367-e590-4d5e-8305-99e188abf71a} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:23.015673 {91652558-16fc-4891-a45b-9a7310a942bc} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:23.042437 {c9a9f69d-c3d2-4291-8f91-988e5ae3a341} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:23.046318 {7fd4a8cd-2bf4-4c36-8ed2-001c4412c1cb} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:23.050564 {851144f0-0da6-4a32-8e34-e46e530b3620} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:23.090745 {a0971454-8b1f-4971-9afa-5aa942caffd7} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - -[milovidov-Pro-P30] 2018.11.28 22:08:23.095710 {fd284264-56e0-47ee-b66c-623eae90caee} [ 27 ] executeQuery: Code: 60, e.displayText() = DB::Exception: Table test.mt doesn't exist., e.what() = DB::Exception (from 127.0.0.1:57420) (in query: SELECT count() FROM test.buffer), Stack trace: - -0. ./clickhouse-server(StackTrace::StackTrace()+0x25) [0xb0493d5] -1. ./clickhouse-server(DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int)+0x4b) [0xb02972b] -2. ./clickhouse-server(DB::Context::getTableImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Exception*) const+0x458) [0xe92f8c8] -3. ./clickhouse-server(DB::Context::getTable(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) const+0x71) [0xe92f361] -4. ./clickhouse-server(DB::StorageBuffer::getQueryProcessingStage(DB::Context const&) const+0x4c) [0xf094f3c] -5. ./clickhouse-server(DB::InterpreterSelectQuery::executeImpl(DB::InterpreterSelectQuery::Pipeline&, std::__1::shared_ptr const&, bool)+0xa5) [0xea86775] -6. ./clickhouse-server(DB::InterpreterSelectQuery::executeWithMultipleStreams()+0x5f) [0xea87fff] -7. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::executeWithMultipleStreams()+0x9a) [0xecc902a] -8. ./clickhouse-server(DB::InterpreterSelectWithUnionQuery::execute()+0x4d) [0xecc92bd] -9. ./clickhouse-server() [0xee65008] -10. ./clickhouse-server(DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum)+0x8e) [0xee6484e] -11. ./clickhouse-server(DB::TCPHandler::runImpl()+0x12c3) [0xb0c2313] -12. ./clickhouse-server(DB::TCPHandler::run()+0x1e) [0xb0c6ebe] -13. ./clickhouse-server(Poco::Net::TCPServerConnection::start()+0x18) [0x10abec08] -14. ./clickhouse-server(Poco::Net::TCPServerDispatcher::run()+0x267) [0x10abf417] -15. ./clickhouse-server(Poco::PooledThread::run()+0x81) [0x10dfcd91] -16. ./clickhouse-server() [0x10df8959] -17. ./clickhouse-server(Poco::ThreadImpl::runnableEntry(void*)+0x70) [0x10df7980] -18. ./clickhouse-server() [0x10dfa3d4] -19. ./clickhouse-server() [0x10dfa35d] -20. ./clickhouse-server(void* std::__1::__thread_proxy >, void* (*)(void*), Poco::ThreadImpl*> >(void*)+0x75) [0x10dfa075] -21. /lib/x86_64-linux-gnu/libpthread.so.0(+0x77fb) [0x7f87222d87fb] -22. /lib/x86_64-linux-gnu/libc.so.6(clone+0x3e) [0x7f8721aa7b5e] - diff --git a/dbms/tests/queries/0_stateless/00763_lock_buffer.sh b/dbms/tests/queries/0_stateless/00763_lock_buffer.sh index 582eee51345..4b8e2931902 100755 --- a/dbms/tests/queries/0_stateless/00763_lock_buffer.sh +++ b/dbms/tests/queries/0_stateless/00763_lock_buffer.sh @@ -17,7 +17,7 @@ function thread1() function thread2() { - seq 1 1000 | sed -r -e 's/.+/SELECT count() FROM test.buffer;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60' + seq 1 1000 | sed -r -e 's/.+/SELECT count() FROM test.buffer;/' | ${CLICKHOUSE_CLIENT} --multiquery --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60' } thread1 & From c5eb6781878f7171905fc187a0868e7137a986f0 Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 29 Nov 2018 00:21:22 +0300 Subject: [PATCH 41/47] PerformanceTest fix --secure (#3690) * fix * PerformanceTest fix --secure * clean --- dbms/programs/benchmark/Benchmark.cpp | 3 +-- .../performance-test/PerformanceTest.cpp | 23 +++++++++++++++---- 2 files changed, 19 insertions(+), 7 deletions(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index d043a91b8b4..2cf8bc6fcfd 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -80,7 +80,6 @@ public: void initialize(Poco::Util::Application & self) { - Poco::Util::Application::initialize(self); std::string home_path; const char * home_path_cstr = getenv("HOME"); if (home_path_cstr) @@ -447,7 +446,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ("json", value()->default_value(""), "write final report to specified file in JSON format") ("host,h", value()->default_value("localhost"), "") ("port", value()->default_value(9000), "") - ("secure", "Use TLS connection") + ("secure,s", "Use TLS connection") ("user", value()->default_value("default"), "") ("password", value()->default_value(""), "") ("database", value()->default_value("default"), "") diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index 4bdb1d64096..7ee6ce7df13 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -30,7 +30,9 @@ #include #include #include +#include #include +#include #ifndef __clang__ #pragma GCC optimize("-fno-var-tracking-assignments") @@ -487,7 +489,7 @@ struct Stats double Stats::avg_rows_speed_precision = 0.001; double Stats::avg_bytes_speed_precision = 0.001; -class PerformanceTest +class PerformanceTest : public Poco::Util::Application { public: using Strings = std::vector; @@ -524,7 +526,19 @@ public: { throw DB::Exception("No tests were specified", DB::ErrorCodes::BAD_ARGUMENTS); } + } + void initialize(Poco::Util::Application & self) + { + std::string home_path; + const char * home_path_cstr = getenv("HOME"); + if (home_path_cstr) + home_path = home_path_cstr; + configReadClient(Poco::Util::Application::instance().config(), home_path); + } + + int main(const std::vector < std::string > & /* args */) + { std::string name; UInt64 version_major; UInt64 version_minor; @@ -1390,7 +1404,7 @@ try ("profiles-file", value()->default_value(""), "Specify a file with global profiles") ("host,h", value()->default_value("localhost"), "") ("port", value()->default_value(9000), "") - ("secure", "Use TLS connection") + ("secure,s", "Use TLS connection") ("database", value()->default_value("default"), "") ("user", value()->default_value("default"), "") ("password", value()->default_value(""), "") @@ -1482,7 +1496,7 @@ try DB::UseSSL use_ssl; - DB::PerformanceTest performanceTest( + DB::PerformanceTest performance_test( options["host"].as(), options["port"].as(), options.count("secure"), @@ -1499,8 +1513,7 @@ try std::move(tests_names_regexp), std::move(skip_names_regexp), timeouts); - - return 0; + return performance_test.run(); } catch (...) { From a75430cbf74fba4a922f189296a99f203940cb9a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 29 Nov 2018 10:58:59 +0300 Subject: [PATCH 42/47] Fix build. --- dbms/programs/benchmark/Benchmark.cpp | 2 +- dbms/programs/performance-test/PerformanceTest.cpp | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 2cf8bc6fcfd..07c65e27f9a 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -78,7 +78,7 @@ public: } - void initialize(Poco::Util::Application & self) + void initialize(Poco::Util::Application & self [[maybe_unused]]) { std::string home_path; const char * home_path_cstr = getenv("HOME"); diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index 7ee6ce7df13..62ba4ffef60 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -551,6 +551,8 @@ public: server_version = ss.str(); processTestsConfigurations(input_files); + + return 0; } private: From 95e42976013fc1f802281804d92b758f9aec89cb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 29 Nov 2018 11:19:14 +0300 Subject: [PATCH 43/47] Fix tests. --- .../queries/0_stateless/00735_conditional.sql | 92 +++++++++---------- ...00763_create_query_as_table_engine_bug.sql | 2 +- 2 files changed, 47 insertions(+), 47 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00735_conditional.sql b/dbms/tests/queries/0_stateless/00735_conditional.sql index aeb90dca7f7..3ab2f8f9e75 100644 --- a/dbms/tests/queries/0_stateless/00735_conditional.sql +++ b/dbms/tests/queries/0_stateless/00735_conditional.sql @@ -12,7 +12,7 @@ SELECT toInt8(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), SELECT toInt8(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt8(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt8(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT toInt8(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt8(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toInt8(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt8(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt8(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -29,7 +29,7 @@ SELECT toInt16(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), SELECT toInt16(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt16(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt16(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT toInt16(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt16(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toInt16(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt16(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt16(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -46,7 +46,7 @@ SELECT toInt32(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), SELECT toInt32(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt32(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt32(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT toInt32(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt32(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toInt32(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt32(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt32(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -63,7 +63,7 @@ SELECT toInt64(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), SELECT toInt64(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt64(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt64(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toInt64(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt64(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toInt64(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt64(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt64(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -80,7 +80,7 @@ SELECT toUInt8(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), SELECT toUInt8(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt8(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt8(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT toUInt8(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt8(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toUInt8(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt8(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt8(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -97,7 +97,7 @@ SELECT toUInt16(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x) SELECT toUInt16(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt16(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt16(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT toUInt16(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt16(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toUInt16(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt16(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt16(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -114,7 +114,7 @@ SELECT toUInt32(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x) SELECT toUInt32(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt32(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt32(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT toUInt32(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt32(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toUInt32(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt32(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt32(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -131,29 +131,29 @@ SELECT toUInt64(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x) SELECT toUInt64(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt64(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt64(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUInt64(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt64(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toUInt64(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt64(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt64(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT toUInt64(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT toUInt64(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } -SELECT toDate(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDate(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDate(0) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDate(0) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDate(0) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDate(0) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDate(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDate(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDate(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDate(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDate(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT toDate(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT toDate(0) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT toDate(0) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT toDate(0) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT toDate(0) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT toDate(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT toDate(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT toDate(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT toDate(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toDate(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toDate(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT toDate(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDate(0) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDate(0) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDate(0) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDate(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT toDate(0) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT toDate(0) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT toDate(0) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toDateTime(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toDateTime(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } @@ -182,7 +182,7 @@ SELECT toUUID(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), SELECT toUUID(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUUID(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUUID(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toUUID(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toUUID(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUUID(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUUID(0) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } @@ -201,7 +201,7 @@ SELECT materialize(toInt8(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, to SELECT materialize(toInt8(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt8(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toInt8(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT materialize(toInt8(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt8(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt8(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt8(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt8(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -218,7 +218,7 @@ SELECT materialize(toInt16(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, t SELECT materialize(toInt16(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt16(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toInt16(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT materialize(toInt16(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt16(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt16(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt16(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt16(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -235,7 +235,7 @@ SELECT materialize(toInt32(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, t SELECT materialize(toInt32(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt32(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toInt32(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT materialize(toInt32(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt32(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt32(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt32(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt32(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -252,7 +252,7 @@ SELECT materialize(toInt64(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, t SELECT materialize(toInt64(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt64(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt64(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toInt64(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt64(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toInt64(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt64(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt64(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -269,7 +269,7 @@ SELECT materialize(toUInt8(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, t SELECT materialize(toUInt8(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt8(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt8(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT materialize(toUInt8(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt8(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt8(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt8(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt8(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -286,7 +286,7 @@ SELECT materialize(toUInt16(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, SELECT materialize(toUInt16(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt16(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt16(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT materialize(toUInt16(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt16(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt16(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt16(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt16(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -303,7 +303,7 @@ SELECT materialize(toUInt32(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, SELECT materialize(toUInt32(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt32(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt32(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT materialize(toUInt32(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt32(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt32(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt32(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt32(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -320,29 +320,29 @@ SELECT materialize(toUInt64(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, SELECT materialize(toUInt64(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt64(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt64(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUInt64(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt64(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUInt64(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt64(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt64(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT materialize(toUInt64(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT materialize(toUInt64(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } -SELECT materialize(toDate(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDate(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDate(0)) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDate(0)) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDate(0)) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDate(0)) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDate(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDate(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDate(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDate(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDate(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT materialize(toDate(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT materialize(toDate(0)) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT materialize(toDate(0)) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT materialize(toDate(0)) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT materialize(toDate(0)) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT materialize(toDate(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT materialize(toDate(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT materialize(toDate(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT materialize(toDate(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toDate(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toDate(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT materialize(toDate(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDate(0)) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDate(0)) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDate(0)) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDate(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT materialize(toDate(0)) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT materialize(toDate(0)) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } +SELECT materialize(toDate(0)) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toDateTime(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toDateTime(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } @@ -371,7 +371,7 @@ SELECT materialize(toUUID(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, to SELECT materialize(toUUID(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUUID(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUUID(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toUUID(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toUUID(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUUID(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUUID(0)) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } diff --git a/dbms/tests/queries/0_stateless/00763_create_query_as_table_engine_bug.sql b/dbms/tests/queries/0_stateless/00763_create_query_as_table_engine_bug.sql index fbaa637fd50..fef87a0771c 100644 --- a/dbms/tests/queries/0_stateless/00763_create_query_as_table_engine_bug.sql +++ b/dbms/tests/queries/0_stateless/00763_create_query_as_table_engine_bug.sql @@ -1,7 +1,7 @@ drop table if exists test.t; drop table if exists test.td; create table test.t (val UInt32) engine = MergeTree order by val; -create table test.td engine = Distributed(test_cluster, 'test', 't') as test.t; +create table test.td engine = Distributed(test_shard_localhost, 'test', 't') as test.t; select engine from system.tables where database = 'test' and name = 'td'; drop table if exists test.t; drop table if exists test.td; From f7f5ad39525222083aa4f411057317b5d9e6db5b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 29 Nov 2018 11:23:27 +0300 Subject: [PATCH 44/47] Fix build. --- dbms/programs/performance-test/PerformanceTest.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index 62ba4ffef60..6a06283a6b8 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -528,7 +528,7 @@ public: } } - void initialize(Poco::Util::Application & self) + void initialize(Poco::Util::Application & self [[maybe_unused]]) { std::string home_path; const char * home_path_cstr = getenv("HOME"); From 837629fbf68a6e2e2f13a87a89cc8c40e2158e07 Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 29 Nov 2018 15:44:57 +0300 Subject: [PATCH 45/47] Better .clang-format and uncrustify.cfg --- .clang-format | 2 ++ uncrustify.cfg | 10 ++++++++-- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/.clang-format b/.clang-format index 7ddeba81288..988c91c8c76 100644 --- a/.clang-format +++ b/.clang-format @@ -29,7 +29,9 @@ AllowShortFunctionsOnASingleLine: InlineOnly AlwaysBreakTemplateDeclarations: true IndentCaseLabels: true SpaceAfterTemplateKeyword: true +SpaceBeforeCpp11BracedList: false SortIncludes: true +IndentPPDirectives: AfterHash IncludeCategories: - Regex: '^<[a-z_]+>' Priority: 1 diff --git a/uncrustify.cfg b/uncrustify.cfg index 70277578854..9fefc1270de 100644 --- a/uncrustify.cfg +++ b/uncrustify.cfg @@ -19,6 +19,7 @@ utf8_force = true sp_arith = force sp_assign = force sp_cpp_lambda_assign = remove +sp_cpp_lambda_paren = remove sp_assign_default = force sp_enum_assign = force sp_enum_colon = force @@ -129,7 +130,7 @@ sp_inside_newop_paren = remove sp_before_tr_emb_cmt = force indent_columns = 4 indent_with_tabs = 0 -indent_namespace = true +indent_namespace = false indent_namespace_limit = 100 indent_class = true indent_ctor_init_leading = 1 @@ -159,6 +160,7 @@ nl_enum_leave_one_liners = true nl_getset_leave_one_liners = true nl_func_leave_one_liners = true nl_cpp_lambda_leave_one_liners = true +nl_cpp_ldef_brace = add nl_if_leave_one_liners = true nl_start_of_file = remove nl_end_of_file = force @@ -230,7 +232,7 @@ eat_blanks_after_open_brace = true eat_blanks_before_close_brace = true nl_remove_extra_newlines = 1 nl_after_return = true -pos_constr_comma = trail_force +pos_constr_comma = lead_break pos_constr_colon = lead_force code_width = 160 ls_func_split_full = true @@ -244,3 +246,7 @@ mod_full_brace_while = remove mod_paren_on_return = remove mod_remove_extra_semicolon = true mod_remove_empty_return = true +align_func_params = true +align_func_params_thresh = 140 +sp_inside_type_brace_init_lst = remove +nl_constr_init_args = add From 1cc69100f1fedb68ce40efaac138858caed52109 Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 29 Nov 2018 17:37:56 +0300 Subject: [PATCH 46/47] Fix macos build --- dbms/src/Dictionaries/XDBCDictionarySource.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/src/Dictionaries/XDBCDictionarySource.cpp b/dbms/src/Dictionaries/XDBCDictionarySource.cpp index 16833420dfe..16101c980ec 100644 --- a/dbms/src/Dictionaries/XDBCDictionarySource.cpp +++ b/dbms/src/Dictionaries/XDBCDictionarySource.cpp @@ -246,6 +246,11 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory) BridgeHelperPtr bridge = std::make_shared>(context, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string")); return std::make_unique(dict_struct, config, config_prefix + ".odbc", sample_block, context, bridge); #else + (void)dict_struct; + (void)config; + (void)config_prefix; + (void)sample_block; + (void)context; throw Exception {"Dictionary source of type `odbc` is disabled because poco library was built without ODBC support.", ErrorCodes::SUPPORT_IS_DISABLED}; #endif From 44ddd28e14f7bd22a028f8df8fd0c48e52fd46dd Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 29 Nov 2018 19:17:29 +0300 Subject: [PATCH 47/47] Auto version update to [18.14.17] [54409] --- dbms/cmake/version.cmake | 8 ++++---- debian/changelog | 4 ++-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index 9f623279b99..276e0db293c 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -2,10 +2,10 @@ set(VERSION_REVISION 54409 CACHE STRING "") set(VERSION_MAJOR 18 CACHE STRING "") set(VERSION_MINOR 14 CACHE STRING "") -set(VERSION_PATCH 9 CACHE STRING "") -set(VERSION_GITHASH 457f8fd495b2812940e69c15ab5b499cd863aae4 CACHE STRING "") -set(VERSION_DESCRIBE v18.14.9-testing CACHE STRING "") -set(VERSION_STRING 18.14.9 CACHE STRING "") +set(VERSION_PATCH 17 CACHE STRING "") +set(VERSION_GITHASH ac2895d769c3dcf070530dec7fcfdcf87bfa852a CACHE STRING "") +set(VERSION_DESCRIBE v18.14.17-testing CACHE STRING "") +set(VERSION_STRING 18.14.17 CACHE STRING "") # end of autochange set(VERSION_EXTRA "" CACHE STRING "") diff --git a/debian/changelog b/debian/changelog index 8c8faa45d4f..805a196370e 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (18.14.9) unstable; urgency=low +clickhouse (18.14.17) unstable; urgency=low * Modified source code - -- Tue, 16 Oct 2018 15:58:16 +0300 + -- Thu, 29 Nov 2018 17:05:34 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index 3c78bb71978..51abca85537 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.14.9 +ARG version=18.14.17 RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index ba65b950fc7..b448e2a4c3a 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.14.9 +ARG version=18.14.17 ARG gosu_ver=1.10 RUN apt-get update \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index 32f0143d678..5018363e2a9 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.14.9 +ARG version=18.14.17 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \