diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 3c8912e4d3d..cbd588a25d1 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -111,6 +111,7 @@ void Connection::disconnect() //LOG_TRACE(log_wrapper.get(), "Disconnecting"); in = nullptr; + last_input_packet_type.reset(); out = nullptr; // can write to socket if (socket) socket->close(); @@ -484,7 +485,26 @@ bool Connection::poll(size_t timeout_microseconds) bool Connection::hasReadBufferPendingData() const { - return static_cast(*in).hasPendingData(); + return last_input_packet_type.has_value() || static_cast(*in).hasPendingData(); +} + + +std::optional Connection::checkPacket(size_t timeout_microseconds) +{ + if (last_input_packet_type.has_value()) + return last_input_packet_type; + + if (hasReadBufferPendingData() || poll(timeout_microseconds)) + { + // LOG_TRACE(log_wrapper.get(), "Receiving packet type"); + UInt64 packet_type; + readVarUInt(packet_type, *in); + + last_input_packet_type.emplace(packet_type); + return last_input_packet_type; + } + + return {}; } @@ -493,7 +513,17 @@ Connection::Packet Connection::receivePacket() try { Packet res; - readVarUInt(res.type, *in); + + if (last_input_packet_type) + { + res.type = *last_input_packet_type; + last_input_packet_type.reset(); + } + else + { + LOG_TRACE(log_wrapper.get(), "Receiving packet type"); + readVarUInt(res.type, *in); + } // LOG_TRACE(log_wrapper.get(), "Receiving packet " << res.type << " " << Protocol::Server::toString(res.type)); diff --git a/dbms/src/Client/Connection.h b/dbms/src/Client/Connection.h index 6a8d0f71781..a7cadecee8d 100644 --- a/dbms/src/Client/Connection.h +++ b/dbms/src/Client/Connection.h @@ -23,6 +23,7 @@ #include #include +#include namespace DB @@ -140,6 +141,9 @@ public: /// Check, if has data in read buffer. bool hasReadBufferPendingData() const; + /// Checks if there is input data in connection and reads packet ID. + std::optional checkPacket(size_t timeout_microseconds = 0); + /// Receive packet from server. Packet receivePacket(); @@ -194,6 +198,7 @@ private: std::unique_ptr socket; std::shared_ptr in; std::shared_ptr out; + std::optional last_input_packet_type; String query_id; Protocol::Compression compression; /// Enable data compression for communication. diff --git a/dbms/src/Client/MultiplexedConnections.cpp b/dbms/src/Client/MultiplexedConnections.cpp index 8fe27ecf7fa..6c534ff53f4 100644 --- a/dbms/src/Client/MultiplexedConnections.cpp +++ b/dbms/src/Client/MultiplexedConnections.cpp @@ -247,6 +247,7 @@ Connection::Packet MultiplexedConnections::receivePacketUnlocked() case Protocol::Server::ProfileInfo: case Protocol::Server::Totals: case Protocol::Server::Extremes: + case Protocol::Server::Log: break; case Protocol::Server::EndOfStream: diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.cpp b/dbms/src/DataStreams/RemoteBlockInputStream.cpp index 7b851ec8469..7baff3de4b4 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockInputStream.cpp @@ -232,6 +232,9 @@ Block RemoteBlockInputStream::readImpl() extremes = packet.block; break; + case Protocol::Server::Log: + break; + default: got_unknown_packet_from_replica = true; throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); diff --git a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp index 51fb62ef5a1..9bc0fa3b101 100644 --- a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp @@ -24,23 +24,31 @@ RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, const */ connection.sendQuery(query, "", QueryProcessingStage::Complete, settings, nullptr); - Connection::Packet packet = connection.receivePacket(); - - if (Protocol::Server::Data == packet.type) + while (true) { - header = packet.block; + Connection::Packet packet = connection.receivePacket(); - if (!header) - throw Exception("Logical error: empty block received as table structure", ErrorCodes::LOGICAL_ERROR); + if (Protocol::Server::Data == packet.type) + { + header = packet.block; + + if (!header) + throw Exception("Logical error: empty block received as table structure", ErrorCodes::LOGICAL_ERROR); + break; + } + else if (Protocol::Server::Exception == packet.type) + { + packet.exception->rethrow(); + break; + } + else if (Protocol::Server::Log == packet.type) + { + /// Do nothing + } + else + throw NetException("Unexpected packet from server (expected Data or Exception, got " + + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); } - else if (Protocol::Server::Exception == packet.type) - { - packet.exception->rethrow(); - return; - } - else - throw NetException("Unexpected packet from server (expected Data or Exception, got " - + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); } @@ -83,18 +91,23 @@ void RemoteBlockOutputStream::writeSuffix() /// Empty block means end of data. connection.sendData(Block()); - /// Receive EndOfStream packet. - Connection::Packet packet = connection.receivePacket(); - - if (Protocol::Server::EndOfStream == packet.type) + /// Wait for EndOfStream or Exception packet, skip Log packets. + while (true) { - /// Do nothing. + Connection::Packet packet = connection.receivePacket(); + + if (Protocol::Server::EndOfStream == packet.type) + break; + else if (Protocol::Server::Exception == packet.type) + packet.exception->rethrow(); + else if (Protocol::Server::Log == packet.type) + { + // Do nothing + } + else + throw NetException("Unexpected packet from server (expected EndOfStream or Exception, got " + + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); } - else if (Protocol::Server::Exception == packet.type) - packet.exception->rethrow(); - else - throw NetException("Unexpected packet from server (expected EndOfStream or Exception, got " - + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); finished = true; } diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 65bb2b3a380..0886fc17455 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -1726,12 +1726,12 @@ private: void thread(Int32 bucket_num, ThreadStatusPtr main_thread) { - setThreadName("MergingAggregtd"); - CurrentThread::attachQueryFromSiblingThread(main_thread); - CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; - try { + setThreadName("MergingAggregtd"); + CurrentThread::attachQueryFromSiblingThreadIfDetached(main_thread); + CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; + /// TODO: add no_more_keys support maybe auto & merged_data = *data[0]; diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index f2057b628c1..1fc010c80ad 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -259,7 +259,7 @@ struct Settings M(SettingBool, log_profile_events, true, "Log query settings into the query_log and query_thread_log.") \ M(SettingBool, log_query_settings, true, "Log query performance statistics into the query_log.") \ M(SettingBool, log_query_threads, true, "Log query threads into system.query_log table.") \ - M(SettingString, server_logs_level, "", "Send internal server logs to client.") + M(SettingString, server_logs_level, "none", "Send internal server logs to client.") #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ TYPE NAME {DEFAULT}; diff --git a/dbms/src/Server/Client.cpp b/dbms/src/Server/Client.cpp index 8e0a140eb4d..b013d195209 100644 --- a/dbms/src/Server/Client.cpp +++ b/dbms/src/Server/Client.cpp @@ -875,7 +875,7 @@ private: /// If structure was received (thus, server has not thrown an exception), /// send our data with that structure. sendData(sample); - receivePacket(); + receiveEndOfQuery(); } } @@ -959,6 +959,11 @@ private: if (!block) break; + + /// Check if server send Log packet + auto packet_type = connection->checkPacket(); + if (packet_type && *packet_type == Protocol::Server::Log) + connection->receivePacket(); } async_block_input->readSuffix(); @@ -1022,7 +1027,7 @@ private: continue; /// If there is no new data, continue checking whether the query was cancelled after a timeout. } - if (!receivePacket()) + if (!receiveAndProcessPacket()) break; } @@ -1033,7 +1038,7 @@ private: /// Receive a part of the result, or progress info or an exception and process it. /// Returns true if one should continue receiving packets. - bool receivePacket() + bool receiveAndProcessPacket() { Connection::Packet packet = connection->receivePacket(); @@ -1081,22 +1086,59 @@ private: /// Receive the block that serves as an example of the structure of table where data will be inserted. bool receiveSampleBlock(Block & out) { - Connection::Packet packet = connection->receivePacket(); - - switch (packet.type) + while (true) { - case Protocol::Server::Data: - out = packet.block; - return true; + Connection::Packet packet = connection->receivePacket(); - case Protocol::Server::Exception: - onException(*packet.exception); - last_exception = std::move(packet.exception); - return false; + switch (packet.type) + { + case Protocol::Server::Data: + out = packet.block; + return true; - default: - throw NetException("Unexpected packet from server (expected Data, got " - + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); + case Protocol::Server::Exception: + onException(*packet.exception); + last_exception = std::move(packet.exception); + return false; + + case Protocol::Server::Log: + onLogData(packet.block); + break; + + default: + throw NetException("Unexpected packet from server (expected Data, Exception or Log, got " + + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); + } + } + } + + + /// Process Log packets, exit when recieve Exception or EndOfStream + bool receiveEndOfQuery() + { + while (true) + { + Connection::Packet packet = connection->receivePacket(); + + switch (packet.type) + { + case Protocol::Server::EndOfStream: + onEndOfStream(); + return true; + + case Protocol::Server::Exception: + onException(*packet.exception); + last_exception = std::move(packet.exception); + return false; + + case Protocol::Server::Log: + onLogData(packet.block); + break; + + default: + throw NetException("Unexpected packet from server (expected Exception, EndOfStream or Log, got " + + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); + } } } diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index df320adc79c..b37e38c696a 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -156,7 +156,7 @@ void TCPHandler::runImpl() /// Should we send internal logs to client? if (client_revision >= DBMS_MIN_REVISION_WITH_SERVER_LOGS - && !query_context.getSettingsRef().server_logs_level.value.empty()) + && query_context.getSettingsRef().server_logs_level.value != "none") { state.logs_queue = std::make_shared(); state.logs_queue->max_priority = Poco::Logger::parseLevel(query_context.getSettingsRef().server_logs_level.value); @@ -885,10 +885,6 @@ void TCPHandler::sendLogs() { Block block = SystemLogsQueue::getSampleBlock(); block.setColumns(std::move(logs_columns)); - block.checkNumberOfRows(); - - std::cerr << "sendLogs: " << block.rows() << " " << block.columns() << "\n"; - sendLogData(block); } } diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 37e1b53c117..3ab93acda99 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -73,7 +73,7 @@ def main(args): base_dir = os.path.abspath(args.queries) tmp_dir = os.path.abspath(args.tmp) - # Keep same default values as in queries/0_stateless/00000_sh_lib.sh + # Keep same default values as in queries/shell_config.sh os.environ.setdefault("CLICKHOUSE_BINARY", args.binary) os.environ.setdefault("CLICKHOUSE_CLIENT", args.client) os.environ.setdefault("CLICKHOUSE_CONFIG", args.configserver) @@ -364,5 +364,7 @@ if __name__ == '__main__': args.client += ' --host=' + os.getenv("CLICKHOUSE_HOST") if os.getenv("CLICKHOUSE_PORT_TCP"): args.client += ' --port=' + os.getenv("CLICKHOUSE_PORT_TCP") + # Will print all warnings and errors + args.client += ' --server_logs_level=warning' main(args) diff --git a/dbms/tests/queries/0_stateless/00155_long_merges.sh b/dbms/tests/queries/0_stateless/00155_long_merges.sh index d711a18a4b9..a5a636a678a 100755 --- a/dbms/tests/queries/0_stateless/00155_long_merges.sh +++ b/dbms/tests/queries/0_stateless/00155_long_merges.sh @@ -49,7 +49,7 @@ function test { $CLICKHOUSE_CLIENT --query="SELECT count() = $MAX, sum(s) = $SUM FROM test.summing" echo $CLICKHOUSE_CLIENT --query="SELECT count() = $SUM, sum(s) = $SUM FROM test.collapsing" - $CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE test.collapsing" + $CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE test.collapsing" --server_logs_file='/dev/null'; $CLICKHOUSE_CLIENT --query="SELECT count() = $MAX, sum(s) = $MAX FROM test.collapsing" echo $CLICKHOUSE_CLIENT --query="SELECT count() = $SUM, sumMerge(s) = $SUM FROM test.aggregating" diff --git a/dbms/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql b/dbms/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql index 4d3265d7903..e4be7b464d2 100644 --- a/dbms/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql +++ b/dbms/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql @@ -1,6 +1,7 @@ CREATE DATABASE IF NOT EXISTS test2; DROP TABLE IF EXISTS test2.mt_buffer; CREATE TABLE test2.mt_buffer (d Date DEFAULT today(), x UInt64) ENGINE = Buffer(test2, mt, 16, 100, 100, 1000000, 1000000, 1000000000, 1000000000); +SET server_logs_level='none'; -- Supress "Destination table test2.mt doesn't exist. Block of data is discarded." INSERT INTO test2.mt_buffer (x) SELECT number AS x FROM system.numbers LIMIT 100000; INSERT INTO test2.mt_buffer (x) SELECT number AS x FROM system.numbers LIMIT 1000000; DROP DATABASE test2; diff --git a/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.sql b/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.sql index 36768985c0f..a2f8537c0a4 100644 --- a/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.sql +++ b/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.sql @@ -1 +1,2 @@ +SET server_logs_level='none'; SELECT count() FROM remote('{127,1}.0.0.{2,3}', system.one) SETTINGS skip_unavailable_shards = 1; diff --git a/dbms/tests/queries/0_stateless/00336_shard_stack_trace.sh b/dbms/tests/queries/0_stateless/00336_shard_stack_trace.sh index 941cd5bbe15..f2a2556d328 100755 --- a/dbms/tests/queries/0_stateless/00336_shard_stack_trace.sh +++ b/dbms/tests/queries/0_stateless/00336_shard_stack_trace.sh @@ -8,6 +8,6 @@ ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?stacktrace=0" -d 'SELECT a' | wc -l [[ $(${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?stacktrace=1" -d 'SELECT a' | wc -l) -gt 3 ]] && echo 'Ok' || echo 'Fail' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT intDiv(number, 0) FROM remote('127.0.0.{2,3}', system.numbers)" | wc -l -$CLICKHOUSE_CLIENT --query="SELECT a" 2>&1 | wc -l -[[ $($CLICKHOUSE_CLIENT --query="SELECT a" --stacktrace 2>&1 | wc -l) -gt 3 ]] && echo 'Ok' || echo 'Fail' -$CLICKHOUSE_CLIENT --query="SELECT intDiv(number, 0) FROM remote('127.0.0.{2,3}', system.numbers)" 2>&1 | wc -l +$CLICKHOUSE_CLIENT --query="SELECT a" --server_logs_file=/dev/null 2>&1 | wc -l +[[ $($CLICKHOUSE_CLIENT --query="SELECT a" --server_logs_file=/dev/null --stacktrace 2>&1 | wc -l) -gt 3 ]] && echo 'Ok' || echo 'Fail' +$CLICKHOUSE_CLIENT --query="SELECT intDiv(number, 0) FROM remote('127.0.0.{2,3}', system.numbers)" --server_logs_file=/dev/null 2>&1 | wc -l diff --git a/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh b/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh index 3b171439b5f..43392107b31 100755 --- a/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh +++ b/dbms/tests/queries/0_stateless/00443_optimize_final_vertical_merge.sh @@ -75,7 +75,7 @@ while [[ `get_num_parts` -ne 1 ]] ; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE $na $CLICKHOUSE_CLIENT -q "ALTER TABLE $name ADD COLUMN n.a Array(String)" $CLICKHOUSE_CLIENT -q "ALTER TABLE $name ADD COLUMN da Array(String) DEFAULT ['def']" -$CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE $name PARTITION 197001 FINAL" +$CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE $name PARTITION 197001 FINAL" --server_logs_file=/dev/null $CLICKHOUSE_CLIENT -q "ALTER TABLE $name MODIFY COLUMN n.a Array(String) DEFAULT ['zzz']" $CLICKHOUSE_CLIENT -q "ALTER TABLE $name MODIFY COLUMN da Array(String) DEFAULT ['zzz']" diff --git a/dbms/tests/queries/0_stateless/00474_readonly_settings.sh b/dbms/tests/queries/0_stateless/00474_readonly_settings.sh index 59f83f43b40..558df9f17b2 100755 --- a/dbms/tests/queries/0_stateless/00474_readonly_settings.sh +++ b/dbms/tests/queries/0_stateless/00474_readonly_settings.sh @@ -6,8 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=0 | grep value $CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=1 | grep value -$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" 2>&1 | grep -o 'value\|Cannot execute SET query in readonly mode.' -$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" 2>&1 | grep -o 'value\|Cannot execute SET query in readonly mode.' +$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o 'value\|Cannot execute SET query in readonly mode.' +$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o 'value\|Cannot execute SET query in readonly mode.' ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" | grep value ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" | grep value diff --git a/dbms/tests/queries/0_stateless/00492_drop_temporary_table.sql b/dbms/tests/queries/0_stateless/00492_drop_temporary_table.sql index dceb4dc2267..3edb44809e9 100644 --- a/dbms/tests/queries/0_stateless/00492_drop_temporary_table.sql +++ b/dbms/tests/queries/0_stateless/00492_drop_temporary_table.sql @@ -2,7 +2,9 @@ DROP TEMPORARY TABLE IF EXISTS temp_tab; CREATE TEMPORARY TABLE temp_tab (number UInt64); INSERT INTO temp_tab SELECT number FROM system.numbers LIMIT 1; SELECT number FROM temp_tab; +SET server_logs_level='none'; DROP TABLE temp_tab; +SET server_logs_level='warning'; CREATE TEMPORARY TABLE temp_tab (number UInt64); SELECT number FROM temp_tab; DROP TEMPORARY TABLE temp_tab; diff --git a/dbms/tests/queries/0_stateless/00550_join_insert_select.sh b/dbms/tests/queries/0_stateless/00550_join_insert_select.sh index 3e78942f989..3fbad66e7c2 100755 --- a/dbms/tests/queries/0_stateless/00550_join_insert_select.sh +++ b/dbms/tests/queries/0_stateless/00550_join_insert_select.sh @@ -20,6 +20,6 @@ INSERT INTO test.test1 SELECT id, name FROM test.test2 ANY LEFT OUTER JOIN test. DROP TABLE test.test1; DROP TABLE test.test2; DROP TABLE test.test3; -" 2>&1 | grep -F "Number of columns doesn't match" | wc -l +" --server_logs_file=/dev/null 2>&1 | grep -F "Number of columns doesn't match" | wc -l $CLICKHOUSE_CLIENT --query="SELECT 1"; diff --git a/dbms/tests/queries/0_stateless/00553_buff_exists_materlized_column.sql b/dbms/tests/queries/0_stateless/00553_buff_exists_materlized_column.sql index 49aff2aa184..9cef126ca49 100644 --- a/dbms/tests/queries/0_stateless/00553_buff_exists_materlized_column.sql +++ b/dbms/tests/queries/0_stateless/00553_buff_exists_materlized_column.sql @@ -15,5 +15,5 @@ INSERT INTO test.nums_buf (n) VALUES (5); SELECT n,m FROM test.nums ORDER BY n; SELECT n,m FROM test.nums_buf ORDER BY n; -DROP TABLE IF EXISTS test.nums; DROP TABLE IF EXISTS test.nums_buf; +DROP TABLE IF EXISTS test.nums; diff --git a/dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh b/dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh index 7db122dcbb7..56e11138e2f 100755 --- a/dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh +++ b/dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh @@ -2,7 +2,7 @@ [ -z "$CLICKHOUSE_CLIENT" ] && CLICKHOUSE_CLIENT="clickhouse-client" -SETTINGS="--compile=1 --min_count_to_compile=0 --max_threads=1 --max_memory_usage=8000000" +SETTINGS="--compile=1 --min_count_to_compile=0 --max_threads=1 --max_memory_usage=8000000 --server_logs_file=/dev/null" output=$($CLICKHOUSE_CLIENT -q "SELECT length(groupArray(number)) FROM (SELECT * FROM system.numbers LIMIT 1000000)" $SETTINGS 2>&1) [[ $? -eq 0 ]] && echo "Expected non-zero RC" diff --git a/dbms/tests/queries/0_stateless/00575_illegal_column_exception_when_drop_depen_column.sh b/dbms/tests/queries/0_stateless/00575_illegal_column_exception_when_drop_depen_column.sh index 08c7f8ef298..d199096815c 100755 --- a/dbms/tests/queries/0_stateless/00575_illegal_column_exception_when_drop_depen_column.sh +++ b/dbms/tests/queries/0_stateless/00575_illegal_column_exception_when_drop_depen_column.sh @@ -10,7 +10,7 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.test;" ${CLICKHOUSE_CLIENT} --query "CREATE TABLE test.test (dt Date DEFAULT now(), id UInt32, id2 UInt32 DEFAULT id + 1) ENGINE = MergeTree(dt, dt, 8192);" ${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test(dt,id) VALUES ('2018-02-22',3), ('2018-02-22',4), ('2018-02-22',5);" ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test.test ORDER BY id;" -echo `${CLICKHOUSE_CLIENT} --query "ALTER TABLE test.test DROP COLUMN id;" 2>&1 | grep -c "$exception_pattern"` +echo `${CLICKHOUSE_CLIENT} --query "ALTER TABLE test.test DROP COLUMN id;" --server_logs_file=/dev/null 2>&1 | grep -c "$exception_pattern"` ${CLICKHOUSE_CLIENT} --query "ALTER TABLE test.test DROP COLUMN id2;" ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test.test ORDER BY id;" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE test.test DROP COLUMN id;" diff --git a/dbms/tests/queries/0_stateless/00595_insert_into_view.sh b/dbms/tests/queries/0_stateless/00595_insert_into_view.sh index 030f1931928..ae3e7d40835 100755 --- a/dbms/tests/queries/0_stateless/00595_insert_into_view.sh +++ b/dbms/tests/queries/0_stateless/00595_insert_into_view.sh @@ -11,7 +11,7 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.test_view;" ${CLICKHOUSE_CLIENT} --query "CREATE TABLE test.test (s String) ENGINE = Log;" ${CLICKHOUSE_CLIENT} --query "CREATE VIEW test.test_view AS SELECT * FROM test.test;" -echo `${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test_view VALUES('test_string');" 2>&1 | grep -c "$exception_pattern"` +(( `${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test_view VALUES('test_string');" 2>&1 | grep -c "$exception_pattern"` >= 1 )) && echo 1 || echo "NO MATCH" ${CLICKHOUSE_CLIENT} --query "INSERT INTO test.test VALUES('test_string');" ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test.test;" diff --git a/dbms/tests/queries/0_stateless/00596_limit_on_expanded_ast.sh b/dbms/tests/queries/0_stateless/00596_limit_on_expanded_ast.sh index 85d13cbdb47..3cbbdefb3c0 100755 --- a/dbms/tests/queries/0_stateless/00596_limit_on_expanded_ast.sh +++ b/dbms/tests/queries/0_stateless/00596_limit_on_expanded_ast.sh @@ -5,6 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) exception_pattern="too big" -${CLICKHOUSE_CLIENT} --max_expanded_ast_elements=500000 --query=" +${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --max_expanded_ast_elements=500000 --query=" select 1 as a, a+a as b, b+b as c, c+c as d, d+d as e, e+e as f, f+f as g, g+g as h, h+h as i, i+i as j, j+j as k, k+k as l, l+l as m, m+m as n, n+n as o, o+o as p, p+p as q, q+q as r, r+r as s, s+s as t, t+t as u, u+u as v, v+v as w, w+w as x, x+x as y, y+y as z " 2>&1 | grep -c "$exception_pattern" diff --git a/dbms/tests/queries/0_stateless/00602_throw_if.sh b/dbms/tests/queries/0_stateless/00602_throw_if.sh index a63c109ffaf..8dae5033978 100755 --- a/dbms/tests/queries/0_stateless/00602_throw_if.sh +++ b/dbms/tests/queries/0_stateless/00602_throw_if.sh @@ -5,5 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) exception_pattern="Value passed to 'throwIf' function is non zero" -${CLICKHOUSE_CLIENT} --query="SELECT throwIf(number = 1000000) FROM system.numbers" 2>&1 | grep -cF "$exception_pattern" -${CLICKHOUSE_CLIENT} --query="SELECT sum(x = 0) FROM (SELECT throwIf(number = 1000000) AS x FROM numbers(1000000))" 2>&1 +${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000000) FROM system.numbers" 2>&1 | grep -cF "$exception_pattern" +${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT sum(x = 0) FROM (SELECT throwIf(number = 1000000) AS x FROM numbers(1000000))" 2>&1 diff --git a/dbms/tests/queries/0_stateless/00626_replace_partition_from_table.sql b/dbms/tests/queries/0_stateless/00626_replace_partition_from_table.sql index daf9950e9d5..397b459142a 100644 --- a/dbms/tests/queries/0_stateless/00626_replace_partition_from_table.sql +++ b/dbms/tests/queries/0_stateless/00626_replace_partition_from_table.sql @@ -43,7 +43,7 @@ SELECT count(), sum(d) FROM test.dst; INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database='test' AND table='dst' AND active AND name LIKE '1_%'; SELECT (max(m) - min(m) > 1) AS new_block_is_generated FROM test_block_numbers; -DROP TABLE test_block_numbers; +DROP TEMPORARY TABLE test_block_numbers; SELECT 'ATTACH FROM'; diff --git a/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sql b/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sql index 304e43d497d..352bf225979 100644 --- a/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sql @@ -55,7 +55,7 @@ SELECT count(), sum(d) FROM test.dst_r2; INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database='test' AND table='dst_r1' AND active AND name LIKE '1_%'; SELECT (max(m) - min(m) > 1) AS new_block_is_generated FROM test_block_numbers; -DROP TABLE test_block_numbers; +DROP TEMPORARY TABLE test_block_numbers; SELECT 'ATTACH FROM'; diff --git a/dbms/tests/queries/shell_config.sh b/dbms/tests/queries/shell_config.sh index 0c12418b602..00b7b67c50a 100644 --- a/dbms/tests/queries/shell_config.sh +++ b/dbms/tests/queries/shell_config.sh @@ -1,6 +1,6 @@ export CLICKHOUSE_BINARY=${CLICKHOUSE_BINARY:="clickhouse"} -export CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:="${CLICKHOUSE_BINARY}-client"} +export CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:="${CLICKHOUSE_BINARY}-client --server_logs_level=warning"} export CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY}-local"} export CLICKHOUSE_CONFIG=${CLICKHOUSE_CONFIG:="/etc/clickhouse-server/config.xml"}