diff --git a/README.md b/README.md index 6538fbee0eb..3de9abdc333 100644 --- a/README.md +++ b/README.md @@ -13,6 +13,6 @@ ClickHouse is an open-source column-oriented database management system that all * You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person. ## Upcoming Events -* [ClickHouse Meetup in Minsk](https://yandex.ru/promo/metrica/clickhouse-minsk) on July 11. +* [ClickHouse Meetup in Saint Petersburg](https://yandex.ru/promo/clickhouse/saint-petersburg-2019) on July 27. * [ClickHouse Meetup in Shenzhen](https://www.huodongxing.com/event/3483759917300) on October 20. * [ClickHouse Meetup in Shanghai](https://www.huodongxing.com/event/4483760336000) on October 27. diff --git a/dbms/programs/performance-test/ConfigPreprocessor.cpp b/dbms/programs/performance-test/ConfigPreprocessor.cpp index 3ea095a5175..5da9650454a 100644 --- a/dbms/programs/performance-test/ConfigPreprocessor.cpp +++ b/dbms/programs/performance-test/ConfigPreprocessor.cpp @@ -14,10 +14,15 @@ std::vector ConfigPreprocessor::processConfig( { std::vector result; - for (const auto & path : paths) + for (const auto & path_str : paths) { - result.emplace_back(XMLConfigurationPtr(new XMLConfiguration(path))); - result.back()->setString("path", Poco::Path(path).absolute().toString()); + auto test = XMLConfigurationPtr(new XMLConfiguration(path_str)); + result.push_back(test); + + const auto path = Poco::Path(path_str); + test->setString("path", path.absolute().toString()); + if (test->getString("name", "") == "") + test->setString("name", path.getBaseName()); } /// Leave tests: diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index c2d8d4f252c..a005fcb5fbb 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -296,6 +297,47 @@ void PerformanceTest::runQueries( break; } } + + if (got_SIGINT) + { + return; + } + + // Pull memory usage data from query log. The log is normally filled in + // background, so we have to flush it synchronously here to see all the + // previous queries. + { + RemoteBlockInputStream flush_log(connection, "system flush logs", + {} /* header */, context); + flush_log.readPrefix(); + while (flush_log.read()); + flush_log.readSuffix(); + } + + for (auto & statistics : statistics_by_run) + { + RemoteBlockInputStream log_reader(connection, + "select memory_usage from system.query_log where type = 2 and query_id = '" + + statistics.query_id + "'", + {} /* header */, context); + + log_reader.readPrefix(); + Block block = log_reader.read(); + if (block.columns() == 0) + { + LOG_WARNING(log, "Query '" << statistics.query_id << "' is not found in query log."); + continue; + } + + assert(block.columns() == 1); + assert(block.getDataTypes()[0]->getName() == "UInt64"); + ColumnPtr column = block.getByPosition(0).column; + assert(column->size() == 1); + StringRef ref = column->getDataAt(0); + assert(ref.size == sizeof(UInt64)); + statistics.memory_usage = *reinterpret_cast(ref.data); + log_reader.readSuffix(); + } } diff --git a/dbms/programs/performance-test/PerformanceTestSuite.cpp b/dbms/programs/performance-test/PerformanceTestSuite.cpp index 5f5c13f3390..91314a0fbff 100644 --- a/dbms/programs/performance-test/PerformanceTestSuite.cpp +++ b/dbms/programs/performance-test/PerformanceTestSuite.cpp @@ -260,15 +260,12 @@ static std::vector getInputFiles(const po::variables_map & options, if (input_files.empty()) throw DB::Exception("Did not find any xml files", DB::ErrorCodes::BAD_ARGUMENTS); - else - LOG_INFO(log, "Found " << input_files.size() << " files"); } else { input_files = options["input-files"].as>(); - LOG_INFO(log, "Found " + std::to_string(input_files.size()) + " input files"); - std::vector collected_files; + std::vector collected_files; for (const std::string & filename : input_files) { fs::path file(filename); @@ -290,6 +287,8 @@ static std::vector getInputFiles(const po::variables_map & options, input_files = std::move(collected_files); } + + LOG_INFO(log, "Found " + std::to_string(input_files.size()) + " input files"); std::sort(input_files.begin(), input_files.end()); return input_files; } diff --git a/dbms/programs/performance-test/ReportBuilder.cpp b/dbms/programs/performance-test/ReportBuilder.cpp index 97d4874ca5d..4aa1933a209 100644 --- a/dbms/programs/performance-test/ReportBuilder.cpp +++ b/dbms/programs/performance-test/ReportBuilder.cpp @@ -157,6 +157,8 @@ std::string ReportBuilder::buildFullReport( runJSON.set("avg_bytes_per_second", statistics.avg_bytes_speed_value); } + runJSON.set("memory_usage", statistics.memory_usage); + run_infos.push_back(runJSON); } } diff --git a/dbms/programs/performance-test/TestStats.h b/dbms/programs/performance-test/TestStats.h index 5d70edc437c..b38ffa7386a 100644 --- a/dbms/programs/performance-test/TestStats.h +++ b/dbms/programs/performance-test/TestStats.h @@ -19,6 +19,7 @@ struct TestStats Stopwatch avg_bytes_speed_watch; bool last_query_was_cancelled = false; + std::string query_id; size_t queries = 0; @@ -49,6 +50,8 @@ struct TestStats size_t number_of_rows_speed_info_batches = 0; size_t number_of_bytes_speed_info_batches = 0; + UInt64 memory_usage = 0; + bool ready = false; // check if a query wasn't interrupted by SIGINT std::string exception; diff --git a/dbms/programs/performance-test/executeQuery.cpp b/dbms/programs/performance-test/executeQuery.cpp index f12808eac36..db82a48d0c1 100644 --- a/dbms/programs/performance-test/executeQuery.cpp +++ b/dbms/programs/performance-test/executeQuery.cpp @@ -2,9 +2,11 @@ #include #include #include +#include namespace DB { + namespace { @@ -36,7 +38,7 @@ void checkFulfilledConditionsAndUpdate( } } -} +} // anonymous namespace void executeQuery( Connection & connection, @@ -47,12 +49,18 @@ void executeQuery( Context & context, const Settings & settings) { + static const std::string query_id_prefix + = Poco::UUIDGenerator::defaultGenerator().create().toString() + "-"; + static int next_query_id = 1; + statistics.watch_per_query.restart(); statistics.last_query_was_cancelled = false; statistics.last_query_rows_read = 0; statistics.last_query_bytes_read = 0; + statistics.query_id = query_id_prefix + std::to_string(next_query_id++); RemoteBlockInputStream stream(connection, query, {}, context, &settings); + stream.setQueryId(statistics.query_id); stream.setProgressCallback( [&](const Progress & value) @@ -70,4 +78,5 @@ void executeQuery( statistics.setTotalTime(); } + } diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index f49e09ba599..e06e953810b 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -331,7 +331,8 @@ struct Settings : public SettingsCollection 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) diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.cpp b/dbms/src/DataStreams/RemoteBlockInputStream.cpp index 740e60ffb09..eb576075f80 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockInputStream.cpp @@ -292,7 +292,7 @@ void RemoteBlockInputStream::sendQuery() established = true; auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings); - multiplexed_connections->sendQuery(timeouts, query, "", stage, &context.getClientInfo(), true); + multiplexed_connections->sendQuery(timeouts, query, query_id, stage, &context.getClientInfo(), true); established = false; sent_query = true; diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.h b/dbms/src/DataStreams/RemoteBlockInputStream.h index 3cef2099030..af8d79c324c 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.h +++ b/dbms/src/DataStreams/RemoteBlockInputStream.h @@ -46,6 +46,11 @@ public: ~RemoteBlockInputStream() override; + /// Set the query_id. For now, used by performance test to later find the query + /// in the server query_log. Must be called before sending the query to the + /// server. + void setQueryId(const std::string& query_id_) { assert(!sent_query); query_id = query_id_; } + /// Specify how we allocate connections on a shard. void setPoolMode(PoolMode pool_mode_) { pool_mode = pool_mode_; } @@ -95,6 +100,7 @@ private: std::unique_ptr multiplexed_connections; const String query; + String query_id = ""; Context context; /// Temporary tables needed to be sent to remote servers diff --git a/dbms/src/Interpreters/InterpreterCheckQuery.cpp b/dbms/src/Interpreters/InterpreterCheckQuery.cpp index 3edcc12fedc..8326a627e6c 100644 --- a/dbms/src/Interpreters/InterpreterCheckQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCheckQuery.cpp @@ -7,6 +7,7 @@ #include #include #include +#include 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(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(), "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(); - 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}}); + for (const auto & check_result : check_results) + { + path_column->insert(check_result.fs_path); + is_passed_column->insert(static_cast(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}}); + } BlockIO res; res.in = std::make_shared(block); diff --git a/dbms/src/Interpreters/InterpreterCheckQuery.h b/dbms/src/Interpreters/InterpreterCheckQuery.h index ca6af4af7bc..c667ca74c22 100644 --- a/dbms/src/Interpreters/InterpreterCheckQuery.h +++ b/dbms/src/Interpreters/InterpreterCheckQuery.h @@ -21,7 +21,6 @@ private: ASTPtr query_ptr; const Context & context; - Block result; }; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 43a507ae902..be9994ece64 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -49,7 +49,17 @@ static void likeStringToBloomFilter( while (cur < data.size() && token_extractor->nextLike(data, &cur, token)) bloom_filter.add(token.c_str(), token.size()); } +/// Unified condition for equals, startsWith and endsWith +bool MergeTreeConditionFullText::createFunctionEqualsCondition(RPNElement & out, const Field & value, const MergeTreeIndexFullText & idx) +{ + out.function = RPNElement::FUNCTION_EQUALS; + out.bloom_filter = std::make_unique( + idx.bloom_filter_size, idx.bloom_filter_hashes, idx.seed); + const auto & str = value.get(); + stringToBloomFilter(str.c_str(), str.size(), idx.token_extractor_func, *out.bloom_filter); + return true; +} MergeTreeIndexGranuleFullText::MergeTreeIndexGranuleFullText(const MergeTreeIndexFullText & index) : IMergeTreeIndexGranule() @@ -129,13 +139,7 @@ const MergeTreeConditionFullText::AtomMap MergeTreeConditionFullText::atom_map "equals", [] (RPNElement & out, const Field & value, const MergeTreeIndexFullText & idx) { - out.function = RPNElement::FUNCTION_EQUALS; - out.bloom_filter = std::make_unique( - idx.bloom_filter_size, idx.bloom_filter_hashes, idx.seed); - - const auto & str = value.get(); - stringToBloomFilter(str.c_str(), str.size(), idx.token_extractor_func, *out.bloom_filter); - return true; + return createFunctionEqualsCondition(out, value, idx); } }, { @@ -168,26 +172,14 @@ const MergeTreeConditionFullText::AtomMap MergeTreeConditionFullText::atom_map "startsWith", [] (RPNElement & out, const Field & value, const MergeTreeIndexFullText & idx) { - out.function = RPNElement::FUNCTION_EQUALS; - out.bloom_filter = std::make_unique( - idx.bloom_filter_size, idx.bloom_filter_hashes, idx.seed); - - const auto & prefix = value.get(); - stringToBloomFilter(prefix.c_str(), prefix.size(), idx.token_extractor_func, *out.bloom_filter); - return true; + return createFunctionEqualsCondition(out, value, idx); } }, { "endsWith", [] (RPNElement & out, const Field & value, const MergeTreeIndexFullText & idx) { - out.function = RPNElement::FUNCTION_EQUALS; - out.bloom_filter = std::make_unique( - idx.bloom_filter_size, idx.bloom_filter_hashes, idx.seed); - - const auto & suffix = value.get(); - stringToBloomFilter(suffix.c_str(), suffix.size(), idx.token_extractor_func, *out.bloom_filter); - return true; + return createFunctionEqualsCondition(out, value, idx); } }, { @@ -196,6 +188,7 @@ const MergeTreeConditionFullText::AtomMap MergeTreeConditionFullText::atom_map { out.function = RPNElement::FUNCTION_MULTI_SEARCH; + /// 2d vector is not needed here but is used because already exists for FUNCTION_IN std::vector> bloom_filters; bloom_filters.emplace_back(); for (const auto & element : value.get()) @@ -405,7 +398,7 @@ bool MergeTreeConditionFullText::atomFromAST( size_t key_arg_pos; /// Position of argument with key column (non-const argument) size_t key_column_num = -1; /// Number of a key column (inside key_column_names array) - std::string func_name = func->name; + const auto & func_name = func->name; if (functionIsInOrGlobalInOperator(func_name) && tryPrepareSetBloomFilter(args, out)) { diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.h b/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.h index af22b83272f..f6230134596 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.h +++ b/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.h @@ -117,6 +117,8 @@ private: bool getKey(const ASTPtr & node, size_t & key_column_num); bool tryPrepareSetBloomFilter(const ASTs & args, RPNElement & out); + static bool createFunctionEqualsCondition(RPNElement & out, const Field & value, const MergeTreeIndexFullText & idx); + static const AtomMap atom_map; const MergeTreeIndexFullText & index; diff --git a/dbms/tests/integration/test_check_table/test.py b/dbms/tests/integration/test_check_table/test.py index 6d53b423896..83df59b44a0 100644 --- a/dbms/tests/integration/test_check_table/test.py +++ b/dbms/tests/integration/test_check_table/test.py @@ -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" diff --git a/dbms/tests/queries/0_stateless/00063_check_query.reference b/dbms/tests/queries/0_stateless/00063_check_query.reference index 9b20cc02e31..6ed281c757a 100644 --- a/dbms/tests/queries/0_stateless/00063_check_query.reference +++ b/dbms/tests/queries/0_stateless/00063_check_query.reference @@ -1,5 +1,2 @@ -N.bin 1 -S.bin 1 -N.bin 1 -S.bin 1 -__marks.mrk 1 +1 +1 diff --git a/dbms/tests/queries/0_stateless/00063_check_query.sql b/dbms/tests/queries/0_stateless/00063_check_query.sql index c9493d19d34..2b806cb3bf2 100644 --- a/dbms/tests/queries/0_stateless/00063_check_query.sql +++ b/dbms/tests/queries/0_stateless/00063_check_query.sql @@ -1,3 +1,5 @@ +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; diff --git a/dbms/tests/queries/0_stateless/00961_check_table.sql b/dbms/tests/queries/0_stateless/00961_check_table.sql index 9752ffc3974..0e0b2c3b483 100644 --- a/dbms/tests/queries/0_stateless/00961_check_table.sql +++ b/dbms/tests/queries/0_stateless/00961_check_table.sql @@ -1,3 +1,4 @@ +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; diff --git a/dbms/tests/queries/1_stateful/00077_log_tinylog_stripelog.reference b/dbms/tests/queries/1_stateful/00077_log_tinylog_stripelog.reference index d8ccbd66592..98f50687de4 100644 --- a/dbms/tests/queries/1_stateful/00077_log_tinylog_stripelog.reference +++ b/dbms/tests/queries/1_stateful/00077_log_tinylog_stripelog.reference @@ -1,17 +1,10 @@ +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 diff --git a/dbms/tests/queries/1_stateful/00077_log_tinylog_stripelog.sql b/dbms/tests/queries/1_stateful/00077_log_tinylog_stripelog.sql index d745203ea6b..2324ad69165 100644 --- a/dbms/tests/queries/1_stateful/00077_log_tinylog_stripelog.sql +++ b/dbms/tests/queries/1_stateful/00077_log_tinylog_stripelog.sql @@ -1,3 +1,5 @@ +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; diff --git a/utils/release/release_lib.sh b/utils/release/release_lib.sh index 75307dfe0b0..4eaa4d4ebbd 100644 --- a/utils/release/release_lib.sh +++ b/utils/release/release_lib.sh @@ -226,12 +226,22 @@ function make_rpm { PACKAGE=clickhouse-server ARCH=all TARGET=noarch - unpack_pack + deb_unpack + mv ${PACKAGE}-$VERSION_FULL-2.spec ${PACKAGE}-$VERSION_FULL-2.spec_tmp + echo "Requires: clickhouse-common-static = $VERSION_FULL-2" >> ${PACKAGE}-$VERSION_FULL-2.spec + echo "Requires: tzdata" >> ${PACKAGE}-$VERSION_FULL-2.spec + echo "Requires: initscripts" >> ${PACKAGE}-$VERSION_FULL-2.spec + cat ${PACKAGE}-$VERSION_FULL-2.spec_tmp >> ${PACKAGE}-$VERSION_FULL-2.spec + rpm_pack PACKAGE=clickhouse-client ARCH=all TARGET=noarch - unpack_pack + deb_unpack + mv ${PACKAGE}-$VERSION_FULL-2.spec ${PACKAGE}-$VERSION_FULL-2.spec_tmp + echo "Requires: clickhouse-common-static = $VERSION_FULL-2" >> ${PACKAGE}-$VERSION_FULL-2.spec + cat ${PACKAGE}-$VERSION_FULL-2.spec_tmp >> ${PACKAGE}-$VERSION_FULL-2.spec + rpm_pack PACKAGE=clickhouse-test ARCH=all diff --git a/website/index.html b/website/index.html index e87223555fd..9b799638c26 100644 --- a/website/index.html +++ b/website/index.html @@ -94,7 +94,7 @@
- Upcoming Meetups: Minsk on July 11 and Shenzhen on October 20 + Upcoming Meetups: Saint Petersburg on July 27 and Shenzhen on October 20