mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Make tests deterministic [#METR-22400].
This commit is contained in:
parent
e387d94275
commit
a50c733345
@ -15,7 +15,8 @@ namespace DB
|
||||
class JSONRowOutputStream : public IRowOutputStream
|
||||
{
|
||||
public:
|
||||
JSONRowOutputStream(WriteBuffer & ostr_, const Block & sample_);
|
||||
JSONRowOutputStream(WriteBuffer & ostr_, const Block & sample_,
|
||||
bool write_statistics_);
|
||||
|
||||
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override;
|
||||
void writeFieldDelimiter() override;
|
||||
@ -66,6 +67,7 @@ protected:
|
||||
|
||||
Progress progress;
|
||||
Stopwatch watch;
|
||||
bool write_statistics;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -15,7 +15,8 @@ namespace DB
|
||||
class XMLRowOutputStream : public IRowOutputStream
|
||||
{
|
||||
public:
|
||||
XMLRowOutputStream(WriteBuffer & ostr_, const Block & sample_);
|
||||
XMLRowOutputStream(WriteBuffer & ostr_, const Block & sample_,
|
||||
bool write_statistics_);
|
||||
|
||||
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override;
|
||||
void writeRowStartDelimiter() override;
|
||||
@ -66,6 +67,7 @@ protected:
|
||||
|
||||
Progress progress;
|
||||
Stopwatch watch;
|
||||
bool write_statistics;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -217,6 +217,10 @@ struct Settings
|
||||
\
|
||||
/** What aggregate function to use for implementation of count(DISTINCT ...) */ \
|
||||
M(SettingString, count_distinct_implementation, "uniq") \
|
||||
\
|
||||
/** Write statistics about read rows, bytes, time elapsed in suitable output formats */ \
|
||||
M(SettingBool, output_format_write_statistics, true) \
|
||||
|
||||
|
||||
/// Всевозможные ограничения на выполнение запроса.
|
||||
Limits limits;
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/DataStreams/NativeBlockInputStream.h>
|
||||
#include <DB/DataStreams/NativeBlockOutputStream.h>
|
||||
#include <DB/DataStreams/TabSeparatedRowInputStream.h>
|
||||
@ -127,13 +128,16 @@ static BlockOutputStreamPtr getOutputImpl(const String & name, WriteBuffer & buf
|
||||
else if (name == "Values")
|
||||
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<ValuesRowOutputStream>(buf));
|
||||
else if (name == "JSON")
|
||||
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<JSONRowOutputStream>(buf, sample));
|
||||
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<JSONRowOutputStream>(buf, sample,
|
||||
context.getSettingsRef().output_format_write_statistics));
|
||||
else if (name == "JSONCompact")
|
||||
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<JSONCompactRowOutputStream>(buf, sample));
|
||||
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<JSONCompactRowOutputStream>(buf, sample,
|
||||
context.getSettingsRef().output_format_write_statistics));
|
||||
else if (name == "JSONEachRow")
|
||||
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<JSONEachRowRowOutputStream>(buf, sample));
|
||||
else if (name == "XML")
|
||||
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<XMLRowOutputStream>(buf, sample));
|
||||
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<XMLRowOutputStream>(buf, sample,
|
||||
context.getSettingsRef().output_format_write_statistics));
|
||||
else if (name == "TSKV")
|
||||
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<TSKVRowOutputStream>(buf, sample));
|
||||
else if (name == "ODBCDriver")
|
||||
|
@ -7,8 +7,8 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
JSONRowOutputStream::JSONRowOutputStream(WriteBuffer & ostr_, const Block & sample_)
|
||||
: dst_ostr(ostr_)
|
||||
JSONRowOutputStream::JSONRowOutputStream(WriteBuffer & ostr_, const Block & sample_, bool write_statistics_)
|
||||
: dst_ostr(ostr_), write_statistics(write_statistics_)
|
||||
{
|
||||
NamesAndTypesList columns(sample_.getColumnsList());
|
||||
fields.assign(columns.begin(), columns.end());
|
||||
@ -113,7 +113,9 @@ void JSONRowOutputStream::writeSuffix()
|
||||
writeIntText(row_count, *ostr);
|
||||
|
||||
writeRowsBeforeLimitAtLeast();
|
||||
writeStatistics();
|
||||
|
||||
if (write_statistics)
|
||||
writeStatistics();
|
||||
|
||||
writeChar('\n', *ostr);
|
||||
writeCString("}\n", *ostr);
|
||||
|
@ -6,8 +6,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
XMLRowOutputStream::XMLRowOutputStream(WriteBuffer & ostr_, const Block & sample_)
|
||||
: dst_ostr(ostr_)
|
||||
XMLRowOutputStream::XMLRowOutputStream(WriteBuffer & ostr_, const Block & sample_, bool write_statistics_)
|
||||
: dst_ostr(ostr_), write_statistics(write_statistics_)
|
||||
{
|
||||
NamesAndTypesList columns(sample_.getColumnsList());
|
||||
fields.assign(columns.begin(), columns.end());
|
||||
@ -119,7 +119,9 @@ void XMLRowOutputStream::writeSuffix()
|
||||
writeCString("</rows>\n", *ostr);
|
||||
|
||||
writeRowsBeforeLimitAtLeast();
|
||||
writeStatistics();
|
||||
|
||||
if (write_statistics)
|
||||
writeStatistics();
|
||||
|
||||
writeCString("</result>\n", *ostr);
|
||||
ostr->next();
|
||||
|
@ -1,3 +1,4 @@
|
||||
SET output_format_write_statistics = 0;
|
||||
SET extremes = 1;
|
||||
SELECT 'Hello, world' FROM (SELECT number FROM system.numbers LIMIT 10) WHERE number < 0
|
||||
FORMAT JSONCompact;
|
||||
|
@ -1,2 +1,4 @@
|
||||
SET output_format_write_statistics = 0;
|
||||
|
||||
SELECT count() FROM (SELECT * FROM system.numbers LIMIT 1000) WHERE 1 IN (SELECT 0 WHERE 0)
|
||||
FORMAT JSON
|
||||
FORMAT JSON;
|
||||
|
@ -1 +1,3 @@
|
||||
SELECT count(), arrayJoin([1, 2, 3]) AS n GROUP BY n WITH TOTALS ORDER BY n LIMIT 1 FORMAT JSON
|
||||
SET output_format_write_statistics = 0;
|
||||
|
||||
SELECT count(), arrayJoin([1, 2, 3]) AS n GROUP BY n WITH TOTALS ORDER BY n LIMIT 1 FORMAT JSON;
|
||||
|
@ -1,3 +1,4 @@
|
||||
SET output_format_write_statistics = 0;
|
||||
SET group_by_two_level_threshold = 1;
|
||||
SELECT ignore(x), count() FROM (SELECT number AS x FROM system.numbers LIMIT 1000 UNION ALL SELECT number AS x FROM system.numbers LIMIT 1000) GROUP BY x WITH TOTALS LIMIT 10 FORMAT JSONCompact;
|
||||
SELECT ignore(x), count() FROM (SELECT number AS x FROM system.numbers LIMIT 1000 UNION ALL SELECT number AS x FROM system.numbers LIMIT 1000) GROUP BY x WITH TOTALS ORDER BY x LIMIT 10 FORMAT JSONCompact;
|
||||
|
@ -1,2 +1,3 @@
|
||||
SET output_format_write_statistics = 0;
|
||||
SELECT arrayJoin(range(100)) AS x FROM remote('127.0.0.1', system.one) WHERE x GLOBAL IN (SELECT toUInt8(arrayJoin(range(100)) + 50)) GROUP BY x ORDER BY x LIMIT 10 FORMAT JSONCompact;
|
||||
SELECT arrayJoin(range(100)) AS x FROM remote('127.0.0.{1,2}', system.one) WHERE x GLOBAL IN (SELECT toUInt8(arrayJoin(range(100)) + 50)) GROUP BY x ORDER BY x LIMIT 10 FORMAT JSONCompact;
|
||||
|
@ -1 +1,2 @@
|
||||
SET output_format_write_statistics = 0;
|
||||
SELECT 'Hello & world' AS s, 'Hello\n<World>', toDateTime('2001-02-03 04:05:06') AS time, arrayMap(x -> toString(x), range(10)) AS arr, (s, time) AS tpl SETTINGS extremes = 1 FORMAT XML;
|
||||
|
@ -1,3 +1,4 @@
|
||||
SET output_format_write_statistics = 0;
|
||||
SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, toDateTime(d) + n AS t, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d, t) AS tuple FROM system.numbers LIMIT 2 FORMAT RowBinary;
|
||||
SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, toDateTime(d) + n AS t, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d, t) AS tuple FROM system.numbers LIMIT 2 FORMAT TabSeparatedWithNamesAndTypes;
|
||||
SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, toDateTime(d) + n AS t, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d, t) AS tuple FROM system.numbers LIMIT 2 FORMAT TabSeparatedRaw;
|
||||
|
@ -1,9 +1,9 @@
|
||||
#!/bin/bash
|
||||
|
||||
curl -sS 'http://localhost:8123/?extremes=1' -d "SELECT 1 AS k, count() GROUP BY k WITH TOTALS";
|
||||
curl -sS 'http://localhost:8123/?extremes=1' -d "SELECT 1234567890123 AS k, count() GROUP BY k WITH TOTALS FORMAT JSON";
|
||||
curl -sS 'http://localhost:8123/?extremes=1' -d "SELECT toFloat32(1.23) AS k, count() GROUP BY k WITH TOTALS FORMAT JSONCompact";
|
||||
curl -sS 'http://localhost:8123/?extremes=1&output_format_write_statistics=0' -d "SELECT 1 AS k, count() GROUP BY k WITH TOTALS";
|
||||
curl -sS 'http://localhost:8123/?extremes=1&output_format_write_statistics=0' -d "SELECT 1234567890123 AS k, count() GROUP BY k WITH TOTALS FORMAT JSON";
|
||||
curl -sS 'http://localhost:8123/?extremes=1&output_format_write_statistics=0' -d "SELECT toFloat32(1.23) AS k, count() GROUP BY k WITH TOTALS FORMAT JSONCompact";
|
||||
|
||||
curl -sS 'http://localhost:8123/?extremes=1' -d "SELECT toDate('2010-01-01') AS k, count() GROUP BY k WITH TOTALS";
|
||||
curl -sS 'http://localhost:8123/?extremes=1' -d "SELECT toDateTime('2010-01-01 01:02:03') AS k, count() GROUP BY k WITH TOTALS FORMAT JSON";
|
||||
curl -sS 'http://localhost:8123/?extremes=1' -d "SELECT 1.1 AS k, count() GROUP BY k WITH TOTALS FORMAT JSONCompact";
|
||||
curl -sS 'http://localhost:8123/?extremes=1&output_format_write_statistics=0' -d "SELECT toDate('2010-01-01') AS k, count() GROUP BY k WITH TOTALS";
|
||||
curl -sS 'http://localhost:8123/?extremes=1&output_format_write_statistics=0' -d "SELECT toDateTime('2010-01-01 01:02:03') AS k, count() GROUP BY k WITH TOTALS FORMAT JSON";
|
||||
curl -sS 'http://localhost:8123/?extremes=1&output_format_write_statistics=0' -d "SELECT 1.1 AS k, count() GROUP BY k WITH TOTALS FORMAT JSONCompact";
|
||||
|
Loading…
Reference in New Issue
Block a user