Make tests deterministic [#METR-22400].

This commit is contained in:
Alexey Milovidov 2016-08-17 23:03:22 +03:00
parent e387d94275
commit a50c733345
14 changed files with 44 additions and 19 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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