未验证 提交 14dc9ad8 编写于 作者: A alexey-milovidov 提交者: GitHub

Merge pull request #6024 from yandex/check_table_old_behaviour

Add settings for check query backward compatibility
......@@ -331,7 +331,8 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.") \
M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.") \
\
M(SettingUInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.")
M(SettingUInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.") \
M(SettingBool, check_query_single_value_result, true, "Return check query result as single 1/0 value")
DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS)
......
......@@ -7,6 +7,7 @@
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnsNumber.h>
#include <Common/typeid_cast.h>
#include <algorithm>
namespace DB
......@@ -42,22 +43,33 @@ BlockIO InterpreterCheckQuery::execute()
StoragePtr table = context.getTable(database_name, table_name);
auto check_results = table->checkData(query_ptr, context);
auto block_structure = getBlockStructure();
auto path_column = block_structure[0].type->createColumn();
auto is_passed_column = block_structure[1].type->createColumn();
auto message_column = block_structure[2].type->createColumn();
for (const auto & check_result : check_results)
Block block;
if (context.getSettingsRef().check_query_single_value_result)
{
path_column->insert(check_result.fs_path);
is_passed_column->insert(static_cast<UInt8>(check_result.success));
message_column->insert(check_result.failure_message);
bool result = std::all_of(check_results.begin(), check_results.end(), [] (const CheckResult & res) { return res.success; });
auto column = ColumnUInt8::create();
column->insertValue(UInt64(result));
block = Block{{std::move(column), std::make_shared<DataTypeUInt8>(), "result"}};
}
else
{
auto block_structure = getBlockStructure();
auto path_column = block_structure[0].type->createColumn();
auto is_passed_column = block_structure[1].type->createColumn();
auto message_column = block_structure[2].type->createColumn();
for (const auto & check_result : check_results)
{
path_column->insert(check_result.fs_path);
is_passed_column->insert(static_cast<UInt8>(check_result.success));
message_column->insert(check_result.failure_message);
}
block = Block({
{std::move(path_column), block_structure[0].type, block_structure[0].name},
{std::move(is_passed_column), block_structure[1].type, block_structure[1].name},
{std::move(message_column), block_structure[2].type, block_structure[2].name}});
}
Block block({
{std::move(path_column), block_structure[0].type, block_structure[0].name},
{std::move(is_passed_column), block_structure[1].type, block_structure[1].name},
{std::move(message_column), block_structure[2].type, block_structure[2].name}});
BlockIO res;
res.in = std::make_shared<OneBlockInputStream>(block);
......
......@@ -21,7 +21,6 @@ private:
ASTPtr query_ptr;
const Context & context;
Block result;
};
}
......@@ -50,35 +50,35 @@ def remove_part_from_disk(node, table, part_name):
def test_check_normal_table_corruption(started_cluster):
node1.query("INSERT INTO non_replicated_mt VALUES (toDate('2019-02-01'), 1, 10), (toDate('2019-02-01'), 2, 12)")
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201902") == "201902_1_1_0\t1\t\n"
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201902", settings={"check_query_single_value_result": 0}) == "201902_1_1_0\t1\t\n"
remove_checksums_on_disk(node1, "non_replicated_mt", "201902_1_1_0")
assert node1.query("CHECK TABLE non_replicated_mt").strip() == "201902_1_1_0\t1\tChecksums recounted and written to disk."
assert node1.query("CHECK TABLE non_replicated_mt", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t1\tChecksums recounted and written to disk."
assert node1.query("SELECT COUNT() FROM non_replicated_mt") == "2\n"
remove_checksums_on_disk(node1, "non_replicated_mt", "201902_1_1_0")
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201902").strip() == "201902_1_1_0\t1\tChecksums recounted and written to disk."
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201902", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t1\tChecksums recounted and written to disk."
assert node1.query("SELECT COUNT() FROM non_replicated_mt") == "2\n"
corrupt_data_part_on_disk(node1, "non_replicated_mt", "201902_1_1_0")
assert node1.query("CHECK TABLE non_replicated_mt").strip() == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 16."
assert node1.query("CHECK TABLE non_replicated_mt", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 16."
assert node1.query("CHECK TABLE non_replicated_mt").strip() == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 16."
assert node1.query("CHECK TABLE non_replicated_mt", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 16."
node1.query("INSERT INTO non_replicated_mt VALUES (toDate('2019-01-01'), 1, 10), (toDate('2019-01-01'), 2, 12)")
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201901") == "201901_2_2_0\t1\t\n"
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "201901_2_2_0\t1\t\n"
corrupt_data_part_on_disk(node1, "non_replicated_mt", "201901_2_2_0")
remove_checksums_on_disk(node1, "non_replicated_mt", "201901_2_2_0")
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201901") == "201901_2_2_0\t0\tCheck of part finished with error: \\'Cannot read all data. Bytes read: 2. Bytes expected: 16.\\'\n"
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "201901_2_2_0\t0\tCheck of part finished with error: \\'Cannot read all data. Bytes read: 2. Bytes expected: 16.\\'\n"
def test_check_replicated_table_simple(started_cluster):
......@@ -90,16 +90,16 @@ def test_check_replicated_table_simple(started_cluster):
assert node1.query("SELECT count() from replicated_mt") == "2\n"
assert node2.query("SELECT count() from replicated_mt") == "2\n"
assert node1.query("CHECK TABLE replicated_mt") == "201902_0_0_0\t1\t\n"
assert node2.query("CHECK TABLE replicated_mt") == "201902_0_0_0\t1\t\n"
assert node1.query("CHECK TABLE replicated_mt", settings={"check_query_single_value_result": 0}) == "201902_0_0_0\t1\t\n"
assert node2.query("CHECK TABLE replicated_mt", settings={"check_query_single_value_result": 0}) == "201902_0_0_0\t1\t\n"
node2.query("INSERT INTO replicated_mt VALUES (toDate('2019-01-02'), 3, 10), (toDate('2019-01-02'), 4, 12)")
node1.query("SYSTEM SYNC REPLICA replicated_mt")
assert node1.query("SELECT count() from replicated_mt") == "4\n"
assert node2.query("SELECT count() from replicated_mt") == "4\n"
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901") == "201901_0_0_0\t1\t\n"
assert node2.query("CHECK TABLE replicated_mt PARTITION 201901") == "201901_0_0_0\t1\t\n"
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "201901_0_0_0\t1\t\n"
assert node2.query("CHECK TABLE replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "201901_0_0_0\t1\t\n"
def test_check_replicated_table_corruption(started_cluster):
......@@ -115,15 +115,15 @@ def test_check_replicated_table_corruption(started_cluster):
part_name = node1.query("SELECT name from system.parts where table = 'replicated_mt' and partition_id = '201901' and active = 1").strip()
corrupt_data_part_on_disk(node1, "replicated_mt", part_name)
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901") == "{p}\t0\tPart {p} looks broken. Removing it and queueing a fetch.\n".format(p=part_name)
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "{p}\t0\tPart {p} looks broken. Removing it and queueing a fetch.\n".format(p=part_name)
node1.query("SYSTEM SYNC REPLICA replicated_mt")
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901") == "{}\t1\t\n".format(part_name)
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "{}\t1\t\n".format(part_name)
assert node1.query("SELECT count() from replicated_mt") == "4\n"
remove_part_from_disk(node2, "replicated_mt", part_name)
assert node2.query("CHECK TABLE replicated_mt PARTITION 201901") == "{p}\t0\tPart {p} looks broken. Removing it and queueing a fetch.\n".format(p=part_name)
assert node2.query("CHECK TABLE replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "{p}\t0\tPart {p} looks broken. Removing it and queueing a fetch.\n".format(p=part_name)
node1.query("SYSTEM SYNC REPLICA replicated_mt")
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901") == "{}\t1\t\n".format(part_name)
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "{}\t1\t\n".format(part_name)
assert node1.query("SELECT count() from replicated_mt") == "4\n"
SET check_query_single_value_result = 1;
DROP TABLE IF EXISTS check_query_tiny_log;
CREATE TABLE check_query_tiny_log (N UInt32, S String) Engine = TinyLog;
......
SET check_query_single_value_result = 0;
DROP TABLE IF EXISTS mt_table;
CREATE TABLE mt_table (d Date, key UInt64, data String) ENGINE = MergeTree() PARTITION BY toYYYYMM(d) ORDER BY key;
......
1
1
1
8873898 12457120258355519194
8873898 12457120258355519194
8873898 12457120258355519194
8873898 12457120258355519194
AdvEngineID.bin 1
CounterID.bin 1
RegionID.bin 1
SearchPhrase.bin 1
UserID.bin 1
__marks.mrk 1
AdvEngineID.bin 1
CounterID.bin 1
RegionID.bin 1
SearchPhrase.bin 1
UserID.bin 1
data.bin 1
index.mrk 1
1
1
1
SET check_query_single_value_result = 1;
DROP TABLE IF EXISTS test.hits_log;
DROP TABLE IF EXISTS test.hits_tinylog;
DROP TABLE IF EXISTS test.hits_stripelog;
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册