Fixed several bug. Trun on --server_logs_level=warning in tests. [#CLICKHOUSE-2910]

Fixed several test to be compatible with --server_logs_level=warning.
This commit is contained in:
Vitaliy Lyudvichenko 2018-06-08 22:50:15 +03:00
parent 99b68d3cf3
commit ed7cd86f09
27 changed files with 167 additions and 71 deletions

View File

@ -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<const ReadBufferFromPocoSocket &>(*in).hasPendingData();
return last_input_packet_type.has_value() || static_cast<const ReadBufferFromPocoSocket &>(*in).hasPendingData();
}
std::optional<UInt64> 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));

View File

@ -23,6 +23,7 @@
#include <Interpreters/TablesStatus.h>
#include <atomic>
#include <optional>
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<UInt64> checkPacket(size_t timeout_microseconds = 0);
/// Receive packet from server.
Packet receivePacket();
@ -194,6 +198,7 @@ private:
std::unique_ptr<Poco::Net::StreamSocket> socket;
std::shared_ptr<ReadBuffer> in;
std::shared_ptr<WriteBuffer> out;
std::optional<UInt64> last_input_packet_type;
String query_id;
Protocol::Compression compression; /// Enable data compression for communication.

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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