From a03f3d9f4180457fc0eee234c6269151ea9a9111 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Tue, 2 Apr 2019 20:34:04 +0300 Subject: [PATCH 01/37] Use python bindings in Kafka tests. --- .../integration/test_storage_kafka/test.py | 73 +++++++++++-------- 1 file changed, 43 insertions(+), 30 deletions(-) diff --git a/dbms/tests/integration/test_storage_kafka/test.py b/dbms/tests/integration/test_storage_kafka/test.py index ed4b6d14d25..b56aae80500 100644 --- a/dbms/tests/integration/test_storage_kafka/test.py +++ b/dbms/tests/integration/test_storage_kafka/test.py @@ -62,22 +62,11 @@ def wait_kafka_is_available(max_retries=50): def kafka_produce(topic, messages): - p = subprocess.Popen(('docker', - 'exec', - '-i', - kafka_id, - '/usr/bin/kafka-console-producer', - '--broker-list', - 'INSIDE://localhost:9092', - '--topic', - topic, - '--sync', - '--message-send-max-retries', - '100'), - stdin=subprocess.PIPE) - p.communicate(messages) - p.stdin.close() - print("Produced {} messages for topic {}".format(len(messages.splitlines()), topic)) + producer = KafkaProducer(bootstrap_servers="localhost:9092") + for message in messages: + producer.send(topic=topic, value=message) + producer.flush() + print ("Produced {} messages for topic {}".format(len(messages), topic)) def kafka_produce_protobuf_messages(topic, start_index, num_messages): @@ -141,9 +130,9 @@ def test_kafka_settings_old_syntax(kafka_cluster): # Don't insert malformed messages since old settings syntax # doesn't support skipping of broken messages. - messages = '' + messages = [] for i in range(50): - messages += json.dumps({'key': i, 'value': i}) + '\n' + messages.append(json.dumps({'key': i, 'value': i})) kafka_produce('old', messages) result = '' @@ -167,18 +156,18 @@ def test_kafka_settings_new_syntax(kafka_cluster): kafka_skip_broken_messages = 1; ''') - messages = '' + messages = [] for i in range(25): - messages += json.dumps({'key': i, 'value': i}) + '\n' + messages.append(json.dumps({'key': i, 'value': i})) kafka_produce('new', messages) # Insert couple of malformed messages. - kafka_produce('new', '}{very_broken_message,\n') - kafka_produce('new', '}another{very_broken_message,\n') + kafka_produce('new', ['}{very_broken_message,']) + kafka_produce('new', ['}another{very_broken_message,']) - messages = '' + messages = [] for i in range(25, 50): - messages += json.dumps({'key': i, 'value': i}) + '\n' + messages.append(json.dumps({'key': i, 'value': i})) kafka_produce('new', messages) result = '' @@ -201,9 +190,9 @@ def test_kafka_csv_with_delimiter(kafka_cluster): kafka_row_delimiter = '\\n'; ''') - messages = '' + messages = [] for i in range(50): - messages += '{i}, {i}\n'.format(i=i) + messages.append('{i}, {i}'.format(i=i)) kafka_produce('csv', messages) result = '' @@ -226,9 +215,9 @@ def test_kafka_tsv_with_delimiter(kafka_cluster): kafka_row_delimiter = '\\n'; ''') - messages = '' + messages = [] for i in range(50): - messages += '{i}\t{i}\n'.format(i=i) + messages.append('{i}\t{i}'.format(i=i)) kafka_produce('tsv', messages) result = '' @@ -239,6 +228,30 @@ def test_kafka_tsv_with_delimiter(kafka_cluster): kafka_check_result(result, True) +def test_kafka_json_without_delimiter(kafka_cluster): + instance.query(''' + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS + kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'json', + kafka_group_name = 'json', + kafka_format = 'JSONEachRow'; + ''') + + messages = '' + for i in range(25): + messages += json.dumps({'key': i, 'value': i}) + '\n' + kafka_produce('json', [messages]) + + result = '' + for i in range(50): + result += instance.query('SELECT * FROM test.kafka') + if kafka_check_result(result): + break + kafka_check_result(result, True) + + def test_kafka_protobuf(kafka_cluster): instance.query(''' CREATE TABLE test.kafka (key UInt64, value String) @@ -282,9 +295,9 @@ def test_kafka_materialized_view(kafka_cluster): SELECT * FROM test.kafka; ''') - messages = '' + messages = [] for i in range(50): - messages += json.dumps({'key': i, 'value': i}) + '\n' + messages.append(json.dumps({'key': i, 'value': i})) kafka_produce('json', messages) for i in range(20): From a4dfa0d58d919c901d6369697e285c8ea5bdb0a7 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Wed, 3 Apr 2019 20:46:54 +0300 Subject: [PATCH 02/37] Fix new test --- dbms/tests/integration/test_storage_kafka/test.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/tests/integration/test_storage_kafka/test.py b/dbms/tests/integration/test_storage_kafka/test.py index b56aae80500..0258d38b8ab 100644 --- a/dbms/tests/integration/test_storage_kafka/test.py +++ b/dbms/tests/integration/test_storage_kafka/test.py @@ -244,6 +244,11 @@ def test_kafka_json_without_delimiter(kafka_cluster): messages += json.dumps({'key': i, 'value': i}) + '\n' kafka_produce('json', [messages]) + messages = '' + for i in range(25, 50): + messages += json.dumps({'key': i, 'value': i}) + '\n' + kafka_produce('json', [messages]) + result = '' for i in range(50): result += instance.query('SELECT * FROM test.kafka') From f154e40ed6427b06773732a7332d9eeca95155e1 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Wed, 17 Apr 2019 19:39:30 +0300 Subject: [PATCH 03/37] Update lz4 --- contrib/lz4 | 2 +- contrib/lz4-cmake/CMakeLists.txt | 5 +---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/contrib/lz4 b/contrib/lz4 index c10863b98e1..5dde00e5d33 160000 --- a/contrib/lz4 +++ b/contrib/lz4 @@ -1 +1 @@ -Subproject commit c10863b98e1503af90616ae99725ecd120265dfb +Subproject commit 5dde00e5d33eb617673f62191781907468beadb7 diff --git a/contrib/lz4-cmake/CMakeLists.txt b/contrib/lz4-cmake/CMakeLists.txt index 382185cc339..192dae31f33 100644 --- a/contrib/lz4-cmake/CMakeLists.txt +++ b/contrib/lz4-cmake/CMakeLists.txt @@ -9,9 +9,6 @@ add_library (lz4 ${LIBRARY_DIR}/xxhash.h ${LIBRARY_DIR}/lz4.h - ${LIBRARY_DIR}/lz4hc.h - ${LIBRARY_DIR}/lz4opt.h) - -target_compile_definitions(lz4 PUBLIC LZ4_DISABLE_DEPRECATE_WARNINGS=1) + ${LIBRARY_DIR}/lz4hc.h) target_include_directories(lz4 PUBLIC ${LIBRARY_DIR}) From 2c7fed45dac551703b23c89bbcc116ed441277b8 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Thu, 18 Apr 2019 14:37:20 +0300 Subject: [PATCH 04/37] try decompress fast --- dbms/src/Compression/CompressionCodecLZ4.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/Compression/CompressionCodecLZ4.cpp b/dbms/src/Compression/CompressionCodecLZ4.cpp index 08553e0920c..deef93c82b7 100644 --- a/dbms/src/Compression/CompressionCodecLZ4.cpp +++ b/dbms/src/Compression/CompressionCodecLZ4.cpp @@ -20,6 +20,7 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_COMPRESS; +extern const int CANNOT_DECOMPRESS; extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; extern const int ILLEGAL_CODEC_PARAMETER; } @@ -47,7 +48,9 @@ UInt32 CompressionCodecLZ4::doCompressData(const char * source, UInt32 source_si void CompressionCodecLZ4::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { - LZ4::decompress(source, dest, source_size, uncompressed_size, lz4_stat); + if (LZ4_decompress_fast(source, dest, uncompressed_size) < 0) + throw Exception("Cannot LZ4_decompress_safe", ErrorCodes::CANNOT_DECOMPRESS); + // LZ4::decompress(source, dest, source_size, uncompressed_size, lz4_stat); } void registerCodecLZ4(CompressionCodecFactory & factory) From 0d70325fb8bba7c924f216a5e99b59615cb5a87e Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Thu, 18 Apr 2019 16:20:47 +0300 Subject: [PATCH 05/37] try decompress fast --- dbms/src/Compression/CompressionCodecLZ4.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Compression/CompressionCodecLZ4.cpp b/dbms/src/Compression/CompressionCodecLZ4.cpp index deef93c82b7..0ee741f0b7d 100644 --- a/dbms/src/Compression/CompressionCodecLZ4.cpp +++ b/dbms/src/Compression/CompressionCodecLZ4.cpp @@ -46,7 +46,7 @@ UInt32 CompressionCodecLZ4::doCompressData(const char * source, UInt32 source_si return LZ4_compress_default(source, dest, source_size, LZ4_COMPRESSBOUND(source_size)); } -void CompressionCodecLZ4::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const +void CompressionCodecLZ4::doDecompressData(const char * source, UInt32, char * dest, UInt32 uncompressed_size) const { if (LZ4_decompress_fast(source, dest, uncompressed_size) < 0) throw Exception("Cannot LZ4_decompress_safe", ErrorCodes::CANNOT_DECOMPRESS); From 7723a63d0a70f5e61e475cfdc4bcc4a6c09c606b Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 18 Apr 2019 17:36:51 +0300 Subject: [PATCH 06/37] Add helper utils to interact with Kafka broker --- utils/kafka/consume.py | 39 +++++++++++++++++++++++ utils/kafka/manage.py | 41 ++++++++++++++++++++++++ utils/kafka/produce.py | 72 ++++++++++++++++++++++++++++++++++++++++++ utils/kafka/status.py | 52 ++++++++++++++++++++++++++++++ 4 files changed, 204 insertions(+) create mode 100755 utils/kafka/consume.py create mode 100755 utils/kafka/manage.py create mode 100755 utils/kafka/produce.py create mode 100755 utils/kafka/status.py diff --git a/utils/kafka/consume.py b/utils/kafka/consume.py new file mode 100755 index 00000000000..ce2a0ef6a96 --- /dev/null +++ b/utils/kafka/consume.py @@ -0,0 +1,39 @@ +#! /usr/bin/env python3 +# -*- coding: utf-8 -*- + +# `pip install …` +import kafka # … kafka-python + +import argparse +from pprint import pprint + + +def main(): + parser = argparse.ArgumentParser(description='Kafka Producer client') + parser.add_argument('--server', type=str, metavar='HOST', default='localhost', + help='Kafka bootstrap-server address') + parser.add_argument('--port', type=int, metavar='PORT', default=9092, + help='Kafka bootstrap-server port') + parser.add_argument('--client', type=str, default='ch-kafka-python', + help='custom client id for this producer') + parser.add_argument('--topic', type=str, required=True, + help='name of Kafka topic to store in') + parser.add_argument('--group', type=str, required=True, + help='name of the consumer group') + + args = parser.parse_args() + config = { + 'bootstrap_servers': f'{args.server}:{args.port}', + 'client_id': args.client, + 'group_id': args.group, + } + client = kafka.KafkaConsumer(**config) + + client.subscribe([args.topic]) + pprint(client.poll(10000)) + client.unsubscribe() + client.close() + + +if __name__ == "__main__": + exit(main()) diff --git a/utils/kafka/manage.py b/utils/kafka/manage.py new file mode 100755 index 00000000000..13bc2fa0388 --- /dev/null +++ b/utils/kafka/manage.py @@ -0,0 +1,41 @@ +#! /usr/bin/env python3 +# -*- coding: utf-8 -*- + +# `pip install …` +import kafka # … kafka-python + +import argparse + + +def main(): + parser = argparse.ArgumentParser(description='Kafka Topic manager') + parser.add_argument('--server', type=str, metavar='HOST', default='localhost', + help='Kafka bootstrap-server address') + parser.add_argument('--port', type=int, metavar='PORT', default=9092, + help='Kafka bootstrap-server port') + parser.add_argument('--client', type=str, default='ch-kafka-python', + help='custom client id for this producer') + + commands = parser.add_mutually_exclusive_group() + commands.add_argument('--create', type=str, metavar='TOPIC', nargs='+', + help='create new topic(s) in the cluster') + commands.add_argument('--delete', type=str, metavar='TOPIC', nargs='+', + help='delete existing topic(s) from the cluster') + + args = parser.parse_args() + config = { + 'bootstrap_servers': f'{args.server}:{args.port}', + 'client_id': args.client, + } + + client = kafka.KafkaAdminClient(**config) + if args.create: + print(client.create_topics(args.create)) + elif args.delete: + print(client.delete_topics(args.delete)) + + client.close() + + +if __name__ == "__main__": + exit(main()) diff --git a/utils/kafka/produce.py b/utils/kafka/produce.py new file mode 100755 index 00000000000..f98b2cbbcdb --- /dev/null +++ b/utils/kafka/produce.py @@ -0,0 +1,72 @@ +#! /usr/bin/env python3 +# -*- coding: utf-8 -*- + +# `pip install …` +import kafka # … kafka-python + +import argparse +from concurrent.futures import ThreadPoolExecutor +import enum +import multiprocessing +import sys + + +class Sync(enum.Enum): + NONE = 'none' + LEAD = 'leader' + ALL = 'all' + + def __str__(self): + return self.value + + def convert(self): + values = { + str(Sync.NONE): '0', + str(Sync.LEAD): '1', + str(Sync.ALL): 'all', + } + return values[self.value] + + +def main(): + parser = argparse.ArgumentParser(description='Produce a single message taken from input') + parser.add_argument('--server', type=str, metavar='HOST', default='localhost', + help='Kafka bootstrap-server address') + parser.add_argument('--port', type=int, metavar='PORT', default=9092, + help='Kafka bootstrap-server port') + parser.add_argument('--client', type=str, default='ch-kafka-python', + help='custom client id for this producer') + parser.add_argument('--topic', type=str, required=True, + help='name of Kafka topic to store in') + parser.add_argument('--retries', type=int, default=0, + help='number of retries to send on failure') + parser.add_argument('--multiply', type=int, default=1, + help='multiplies incoming string many times') + parser.add_argument('--repeat', type=int, default=1, + help='send same (multiplied) message many times') + + args = parser.parse_args() + config = { + 'bootstrap_servers': f'{args.server}:{args.port}', + 'client_id': args.client, + 'retries': args.retries, + } + client = kafka.KafkaProducer(**config) + + message = sys.stdin.buffer.read() * args.multiply + + def send(num): + client.send(topic=args.topic, value=message) + print(f'iteration {num}: sent a message multiplied {args.multiply} times') + + pool = ThreadPoolExecutor(max_workers=multiprocessing.cpu_count()) + for num in range(args.repeat): + pool.submit(send, num) + pool.shutdown() + + client.flush() + client.close() + + +if __name__ == "__main__": + exit(main()) diff --git a/utils/kafka/status.py b/utils/kafka/status.py new file mode 100755 index 00000000000..8331a056dff --- /dev/null +++ b/utils/kafka/status.py @@ -0,0 +1,52 @@ +#! /usr/bin/env python3 +# -*- coding: utf-8 -*- + +# `pip install …` +import kafka # … kafka-python + +import argparse +from pprint import pprint + + +def main(): + parser = argparse.ArgumentParser(description='Kafka client to get groups and topics status') + parser.add_argument('--server', type=str, metavar='HOST', default='localhost', + help='Kafka bootstrap-server address') + parser.add_argument('--port', type=int, metavar='PORT', default=9092, + help='Kafka bootstrap-server port') + parser.add_argument('--client', type=str, default='ch-kafka-python', + help='custom client id for this producer') + + args = parser.parse_args() + config = { + 'bootstrap_servers': f'{args.server}:{args.port}', + 'client_id': args.client, + } + + client = kafka.KafkaAdminClient(**config) + consumer = kafka.KafkaConsumer(**config) + cluster = client._client.cluster + + topics = cluster.topics() + for topic in topics: + print(f'Topic "{topic}":', end='') + for partition in cluster.partitions_for_topic(topic): + tp = kafka.TopicPartition(topic, partition) + print(f' {partition} (begin: {consumer.beginning_offsets([tp])[tp]}, end: {consumer.end_offsets([tp])[tp]})', end='') + print() + + groups = client.list_consumer_groups() + for group in groups: + print(f'Group "{group[0]}" ({group[1]}):') + + consumer = kafka.KafkaConsumer(**config, group_id=group[0]) + offsets = client.list_consumer_group_offsets(group[0]) + for topic, offset in offsets.items(): + print(f'\t{topic.topic}[{topic.partition}]: {consumer.beginning_offsets([topic])[topic]}, {offset.offset}, {consumer.end_offsets([topic])[topic]}') + consumer.close() + + client.close() + + +if __name__ == "__main__": + exit(main()) From 719e0c5731913d74bbf510deba1fe74926aaa6ce Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 18 Apr 2019 17:45:23 +0300 Subject: [PATCH 07/37] minor test improvement --- .../0_stateless/00722_inner_join.reference | 19 +++++++--- .../queries/0_stateless/00722_inner_join.sql | 35 ++++++++++++------- 2 files changed, 37 insertions(+), 17 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00722_inner_join.reference b/dbms/tests/queries/0_stateless/00722_inner_join.reference index c482ca7ba9d..b5e8a77a20d 100644 --- a/dbms/tests/queries/0_stateless/00722_inner_join.reference +++ b/dbms/tests/queries/0_stateless/00722_inner_join.reference @@ -19,10 +19,21 @@ ┌─database─┬─name─┐ │ system │ one │ └──────────┴──────┘ -one -system one -system one -test one +┌─db.x───┬─name─┐ +│ system │ one │ +└────────┴──────┘ +┌─db.name─┬─name─┐ +│ system │ one │ +└─────────┴──────┘ +┌─db.name─┬─name─┐ +│ system │ one │ +└─────────┴──────┘ +┌─database─┬─name─┐ +│ system │ one │ +└──────────┴──────┘ +┌─database─┬─name─┐ +│ system │ one │ +└──────────┴──────┘ 2 2 2 diff --git a/dbms/tests/queries/0_stateless/00722_inner_join.sql b/dbms/tests/queries/0_stateless/00722_inner_join.sql index aa1e3674579..5cc0983755b 100644 --- a/dbms/tests/queries/0_stateless/00722_inner_join.sql +++ b/dbms/tests/queries/0_stateless/00722_inner_join.sql @@ -1,6 +1,6 @@ CREATE DATABASE IF NOT EXISTS test; -DROP TABLE IF EXISTS one; -CREATE TABLE one(dummy UInt8) ENGINE = Memory; +DROP TABLE IF EXISTS test.one; +CREATE TABLE test.one(dummy UInt8) ENGINE = Memory; SELECT database, t.name FROM system.tables AS t @@ -48,25 +48,34 @@ SELECT database, t.name WHERE db.name = 'system' AND t.name = 'one' FORMAT PrettyCompactNoEscapes; ---SELECT db.name, t.name --- FROM (SELECT name, database AS x FROM system.tables) AS t --- JOIN (SELECT name AS x FROM system.databases) AS db USING x --- WHERE x = 'system' AND t.name = 'one' --- FORMAT PrettyCompactNoEscapes; +SELECT db.x, t.name + FROM (SELECT name, database AS x FROM system.tables) AS t + JOIN (SELECT name AS x FROM system.databases) AS db USING x + WHERE x = 'system' AND t.name = 'one' + FORMAT PrettyCompactNoEscapes; -SELECT t.name --, db.name +SELECT db.name, t.name FROM (SELECT name, database FROM system.tables WHERE name = 'one') AS t - JOIN (SELECT name FROM system.databases WHERE name = 'system') AS db ON t.database = db.name; + JOIN (SELECT name FROM system.databases WHERE name = 'system') AS db ON t.database = db.name + FORMAT PrettyCompactNoEscapes; SELECT db.name, t.name FROM system.tables AS t JOIN (SELECT * FROM system.databases WHERE name = 'system') AS db ON t.database = db.name - WHERE t.name = 'one'; + WHERE t.name = 'one' + FORMAT PrettyCompactNoEscapes; -SELECT database, t.name +SELECT t.database, t.name FROM system.tables AS t JOIN (SELECT name, name AS database FROM system.databases) AS db ON t.database = db.name - WHERE t.name = 'one'; + WHERE t.database = 'system' AND t.name = 'one' + FORMAT PrettyCompactNoEscapes; + +SELECT t.database, t.name + FROM system.tables t + ANY LEFT JOIN (SELECT 'system' AS base, 'one' AS name) db USING name + WHERE t.database = db.base + FORMAT PrettyCompactNoEscapes; SELECT count(t.database) FROM (SELECT * FROM system.tables WHERE name = 'one') AS t @@ -82,4 +91,4 @@ SELECT count() JOIN system.databases AS db ON db.name = t.database WHERE t.name = 'one'; -DROP TABLE one; +DROP TABLE test.one; From dce48e93639b79bc8ae50c7906af4411ab24c8f0 Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 18 Apr 2019 19:50:38 +0300 Subject: [PATCH 08/37] one more test for Unknown identifier --- .../00860_unknown_identifier_bug.reference | 2 + .../00860_unknown_identifier_bug.sql | 41 +++++++++++++++++++ 2 files changed, 43 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00860_unknown_identifier_bug.reference create mode 100644 dbms/tests/queries/0_stateless/00860_unknown_identifier_bug.sql diff --git a/dbms/tests/queries/0_stateless/00860_unknown_identifier_bug.reference b/dbms/tests/queries/0_stateless/00860_unknown_identifier_bug.reference new file mode 100644 index 00000000000..d574676cc2a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00860_unknown_identifier_bug.reference @@ -0,0 +1,2 @@ +1 Created 1 +2 Created 2 diff --git a/dbms/tests/queries/0_stateless/00860_unknown_identifier_bug.sql b/dbms/tests/queries/0_stateless/00860_unknown_identifier_bug.sql new file mode 100644 index 00000000000..cd5f45fd3fc --- /dev/null +++ b/dbms/tests/queries/0_stateless/00860_unknown_identifier_bug.sql @@ -0,0 +1,41 @@ +USE test; + +DROP TABLE IF EXISTS appointment_events; +CREATE TABLE appointment_events +( + _appointment_id UInt32, + _id String, + _status String, + _set_by_id String, + _company_id String, + _client_id String, + _type String, + _at String, + _vacancy_id String, + _set_at UInt32, + _job_requisition_id String +) ENGINE = Memory; + +INSERT INTO appointment_events (_appointment_id, _set_at, _status) values (1, 1, 'Created'), (2, 2, 'Created'); + +SELECT A._appointment_id, + A._id, + A._status, + A._set_by_id, + A._company_id, + A._client_id, + A._type, + A._at, + A._vacancy_id, + A._set_at, + A._job_requisition_id +FROM test.appointment_events A ANY +LEFT JOIN + (SELECT _appointment_id, + MAX(_set_at) AS max_set_at + FROM test.appointment_events + WHERE _status in ('Created', 'Transferred') + GROUP BY _appointment_id ) B USING _appointment_id +WHERE A._set_at = B.max_set_at; + +DROP TABLE appointment_events; From 500b88d19d5411192d743eb324dfacb108a16d9e Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Fri, 19 Apr 2019 10:46:00 +0300 Subject: [PATCH 09/37] Update lz4 to latest --- contrib/lz4 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/lz4 b/contrib/lz4 index 5dde00e5d33..780aac520b6 160000 --- a/contrib/lz4 +++ b/contrib/lz4 @@ -1 +1 @@ -Subproject commit 5dde00e5d33eb617673f62191781907468beadb7 +Subproject commit 780aac520b69d6369f4e3995624c37e56d75498d From e34e396ff90c25b8eb96c39b175e6898a41725f3 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Fri, 19 Apr 2019 12:10:54 +0300 Subject: [PATCH 10/37] Disable warnings --- contrib/lz4-cmake/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/contrib/lz4-cmake/CMakeLists.txt b/contrib/lz4-cmake/CMakeLists.txt index 192dae31f33..25cceaa4574 100644 --- a/contrib/lz4-cmake/CMakeLists.txt +++ b/contrib/lz4-cmake/CMakeLists.txt @@ -11,4 +11,6 @@ add_library (lz4 ${LIBRARY_DIR}/lz4.h ${LIBRARY_DIR}/lz4hc.h) +target_compile_definitions(lz4 PUBLIC LZ4_DISABLE_DEPRECATE_WARNINGS=1) + target_include_directories(lz4 PUBLIC ${LIBRARY_DIR}) From c7731b32f1da3fd931b27d23911bff10878f184b Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Fri, 19 Apr 2019 15:34:56 +0300 Subject: [PATCH 11/37] Try safe version --- dbms/src/Compression/CompressionCodecLZ4.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Compression/CompressionCodecLZ4.cpp b/dbms/src/Compression/CompressionCodecLZ4.cpp index 0ee741f0b7d..5a3085d54f9 100644 --- a/dbms/src/Compression/CompressionCodecLZ4.cpp +++ b/dbms/src/Compression/CompressionCodecLZ4.cpp @@ -46,9 +46,9 @@ UInt32 CompressionCodecLZ4::doCompressData(const char * source, UInt32 source_si return LZ4_compress_default(source, dest, source_size, LZ4_COMPRESSBOUND(source_size)); } -void CompressionCodecLZ4::doDecompressData(const char * source, UInt32, char * dest, UInt32 uncompressed_size) const +void CompressionCodecLZ4::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { - if (LZ4_decompress_fast(source, dest, uncompressed_size) < 0) + if (LZ4_decompress_safe(source, dest, source_size, uncompressed_size) < 0) throw Exception("Cannot LZ4_decompress_safe", ErrorCodes::CANNOT_DECOMPRESS); // LZ4::decompress(source, dest, source_size, uncompressed_size, lz4_stat); } From cde69ba90037ca6551e250270e662bc119cd7b84 Mon Sep 17 00:00:00 2001 From: never lee Date: Fri, 19 Apr 2019 20:49:16 +0800 Subject: [PATCH 12/37] fix format of docs/zh/operations/table_engines/mergetree.md (#5058) --- docs/zh/operations/table_engines/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/table_engines/mergetree.md b/docs/zh/operations/table_engines/mergetree.md index ac671887dea..953f5128337 100644 --- a/docs/zh/operations/table_engines/mergetree.md +++ b/docs/zh/operations/table_engines/mergetree.md @@ -127,7 +127,7 @@ ClickHouse 会为每个数据分片创建一个索引文件,索引文件包含 你可以只用一单一大表并不断地一块块往里面加入数据 – `MergeTree` 引擎的就是为了这样的场景。 -## 主键和索引在查询中的表现 {#primary-keys-and-indexes-in-queriesko +## 主键和索引在查询中的表现 {#primary-keys-and-indexes-in-queries} 我们以 `(CounterID, Date)` 以主键。排序好的索引的图示会是下面这样: From ee1a843609b379f9bd3b97705bff6510ca29e06c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Apr 2019 23:21:17 +0300 Subject: [PATCH 13/37] COWPtr: improvements (C++ Russia 2019, suggested by Alexey Malov) --- dbms/src/Columns/ColumnAggregateFunction.h | 4 +-- dbms/src/Columns/ColumnArray.h | 6 ++-- dbms/src/Columns/ColumnConst.h | 4 +-- dbms/src/Columns/ColumnDecimal.h | 4 +-- dbms/src/Columns/ColumnFixedString.h | 4 +-- dbms/src/Columns/ColumnFunction.h | 4 +-- dbms/src/Columns/ColumnLowCardinality.h | 6 ++-- dbms/src/Columns/ColumnNothing.h | 4 +-- dbms/src/Columns/ColumnNullable.h | 6 ++-- dbms/src/Columns/ColumnSet.h | 4 +-- dbms/src/Columns/ColumnString.h | 4 +-- dbms/src/Columns/ColumnTuple.h | 6 ++-- dbms/src/Columns/ColumnUnique.h | 4 +-- dbms/src/Columns/ColumnVector.h | 4 +-- dbms/src/Columns/IColumn.h | 8 ++--- dbms/src/Common/{COWPtr.h => COW.h} | 36 +++++++++++----------- dbms/src/Common/tests/cow_columns.cpp | 10 +++--- dbms/src/Common/tests/cow_compositions.cpp | 14 ++++----- dbms/src/Core/iostream_debug_helpers.cpp | 2 +- dbms/src/DataTypes/IDataType.h | 6 ++-- 20 files changed, 70 insertions(+), 70 deletions(-) rename dbms/src/Common/{COWPtr.h => COW.h} (90%) diff --git a/dbms/src/Columns/ColumnAggregateFunction.h b/dbms/src/Columns/ColumnAggregateFunction.h index a028a95d68c..fdfaec93c88 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.h +++ b/dbms/src/Columns/ColumnAggregateFunction.h @@ -43,13 +43,13 @@ using Arenas = std::vector; * specifying which individual values should be destroyed and which ones should not. * Clearly, this method would have a substantially non-zero price. */ -class ColumnAggregateFunction final : public COWPtrHelper +class ColumnAggregateFunction final : public COWHelper { public: using Container = PaddedPODArray; private: - friend class COWPtrHelper; + friend class COWHelper; /// Memory pools. Aggregate states are allocated from them. Arenas arenas; diff --git a/dbms/src/Columns/ColumnArray.h b/dbms/src/Columns/ColumnArray.h index 85df5550d4e..f3f7f1f4a1a 100644 --- a/dbms/src/Columns/ColumnArray.h +++ b/dbms/src/Columns/ColumnArray.h @@ -13,10 +13,10 @@ namespace DB * In memory, it is represented as one column of a nested type, whose size is equal to the sum of the sizes of all arrays, * and as an array of offsets in it, which allows you to get each element. */ -class ColumnArray final : public COWPtrHelper +class ColumnArray final : public COWHelper { private: - friend class COWPtrHelper; + friend class COWHelper; /** Create an array column with specified values and offsets. */ ColumnArray(MutableColumnPtr && nested_column, MutableColumnPtr && offsets_column); @@ -30,7 +30,7 @@ public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. */ - using Base = COWPtrHelper; + using Base = COWHelper; static Ptr create(const ColumnPtr & nested_column, const ColumnPtr & offsets_column) { diff --git a/dbms/src/Columns/ColumnConst.h b/dbms/src/Columns/ColumnConst.h index 87371895840..05a9562e549 100644 --- a/dbms/src/Columns/ColumnConst.h +++ b/dbms/src/Columns/ColumnConst.h @@ -18,10 +18,10 @@ namespace ErrorCodes /** ColumnConst contains another column with single element, * but looks like a column with arbitrary amount of same elements. */ -class ColumnConst final : public COWPtrHelper +class ColumnConst final : public COWHelper { private: - friend class COWPtrHelper; + friend class COWHelper; WrappedPtr data; size_t s; diff --git a/dbms/src/Columns/ColumnDecimal.h b/dbms/src/Columns/ColumnDecimal.h index 4a3c6153947..db762e5f4c3 100644 --- a/dbms/src/Columns/ColumnDecimal.h +++ b/dbms/src/Columns/ColumnDecimal.h @@ -55,13 +55,13 @@ private: /// A ColumnVector for Decimals template -class ColumnDecimal final : public COWPtrHelper> +class ColumnDecimal final : public COWHelper> { static_assert(IsDecimalNumber); private: using Self = ColumnDecimal; - friend class COWPtrHelper; + friend class COWHelper; public: using Container = DecimalPaddedPODArray; diff --git a/dbms/src/Columns/ColumnFixedString.h b/dbms/src/Columns/ColumnFixedString.h index 1f79594b459..2b06d19d7ca 100644 --- a/dbms/src/Columns/ColumnFixedString.h +++ b/dbms/src/Columns/ColumnFixedString.h @@ -13,10 +13,10 @@ namespace DB /** A column of values of "fixed-length string" type. * If you insert a smaller string, it will be padded with zero bytes. */ -class ColumnFixedString final : public COWPtrHelper +class ColumnFixedString final : public COWHelper { public: - friend class COWPtrHelper; + friend class COWHelper; using Chars = PaddedPODArray; diff --git a/dbms/src/Columns/ColumnFunction.h b/dbms/src/Columns/ColumnFunction.h index 8d52110c9ac..571123ae892 100644 --- a/dbms/src/Columns/ColumnFunction.h +++ b/dbms/src/Columns/ColumnFunction.h @@ -15,10 +15,10 @@ namespace DB /** A column containing a lambda expression. * Behaves like a constant-column. Contains an expression, but not input or output data. */ -class ColumnFunction final : public COWPtrHelper +class ColumnFunction final : public COWHelper { private: - friend class COWPtrHelper; + friend class COWHelper; ColumnFunction(size_t size, FunctionBasePtr function, const ColumnsWithTypeAndName & columns_to_capture); diff --git a/dbms/src/Columns/ColumnLowCardinality.h b/dbms/src/Columns/ColumnLowCardinality.h index 91f5337b633..d36b91b0c40 100644 --- a/dbms/src/Columns/ColumnLowCardinality.h +++ b/dbms/src/Columns/ColumnLowCardinality.h @@ -14,9 +14,9 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -class ColumnLowCardinality final : public COWPtrHelper +class ColumnLowCardinality final : public COWHelper { - friend class COWPtrHelper; + friend class COWHelper; ColumnLowCardinality(MutableColumnPtr && column_unique, MutableColumnPtr && indexes, bool is_shared = false); ColumnLowCardinality(const ColumnLowCardinality & other) = default; @@ -25,7 +25,7 @@ public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. */ - using Base = COWPtrHelper; + using Base = COWHelper; static Ptr create(const ColumnPtr & column_unique_, const ColumnPtr & indexes_, bool is_shared = false) { return ColumnLowCardinality::create(column_unique_->assumeMutable(), indexes_->assumeMutable(), is_shared); diff --git a/dbms/src/Columns/ColumnNothing.h b/dbms/src/Columns/ColumnNothing.h index 0c9f843a454..691143e2c15 100644 --- a/dbms/src/Columns/ColumnNothing.h +++ b/dbms/src/Columns/ColumnNothing.h @@ -6,10 +6,10 @@ namespace DB { -class ColumnNothing final : public COWPtrHelper +class ColumnNothing final : public COWHelper { private: - friend class COWPtrHelper; + friend class COWHelper; ColumnNothing(size_t s_) { diff --git a/dbms/src/Columns/ColumnNullable.h b/dbms/src/Columns/ColumnNullable.h index a66979c0683..43a6256b1a5 100644 --- a/dbms/src/Columns/ColumnNullable.h +++ b/dbms/src/Columns/ColumnNullable.h @@ -20,10 +20,10 @@ using ConstNullMapPtr = const NullMap *; /// over a bitmap because columns are usually stored on disk as compressed /// files. In this regard, using a bitmap instead of a byte map would /// greatly complicate the implementation with little to no benefits. -class ColumnNullable final : public COWPtrHelper +class ColumnNullable final : public COWHelper { private: - friend class COWPtrHelper; + friend class COWHelper; ColumnNullable(MutableColumnPtr && nested_column_, MutableColumnPtr && null_map_); ColumnNullable(const ColumnNullable &) = default; @@ -32,7 +32,7 @@ public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. */ - using Base = COWPtrHelper; + using Base = COWHelper; static Ptr create(const ColumnPtr & nested_column_, const ColumnPtr & null_map_) { return ColumnNullable::create(nested_column_->assumeMutable(), null_map_->assumeMutable()); diff --git a/dbms/src/Columns/ColumnSet.h b/dbms/src/Columns/ColumnSet.h index 3ed6c4c4e84..83ab6de5578 100644 --- a/dbms/src/Columns/ColumnSet.h +++ b/dbms/src/Columns/ColumnSet.h @@ -14,10 +14,10 @@ using ConstSetPtr = std::shared_ptr; * Behaves like a constant-column (because the set is one, not its own for each line). * This column has a nonstandard value, so it can not be obtained via a normal interface. */ -class ColumnSet final : public COWPtrHelper +class ColumnSet final : public COWHelper { private: - friend class COWPtrHelper; + friend class COWHelper; ColumnSet(size_t s_, const ConstSetPtr & data_) : data(data_) { s = s_; } ColumnSet(const ColumnSet &) = default; diff --git a/dbms/src/Columns/ColumnString.h b/dbms/src/Columns/ColumnString.h index 486e6b1fd44..398077ce964 100644 --- a/dbms/src/Columns/ColumnString.h +++ b/dbms/src/Columns/ColumnString.h @@ -18,14 +18,14 @@ namespace DB /** Column for String values. */ -class ColumnString final : public COWPtrHelper +class ColumnString final : public COWHelper { public: using Char = UInt8; using Chars = PaddedPODArray; private: - friend class COWPtrHelper; + friend class COWHelper; /// Maps i'th position to offset to i+1'th element. Last offset maps to the end of all chars (is the size of all chars). Offsets offsets; diff --git a/dbms/src/Columns/ColumnTuple.h b/dbms/src/Columns/ColumnTuple.h index 376c099c1dc..65dd19fc6da 100644 --- a/dbms/src/Columns/ColumnTuple.h +++ b/dbms/src/Columns/ColumnTuple.h @@ -12,10 +12,10 @@ namespace DB * Mixed constant/non-constant columns is prohibited in tuple * for implementation simplicity. */ -class ColumnTuple final : public COWPtrHelper +class ColumnTuple final : public COWHelper { private: - friend class COWPtrHelper; + friend class COWHelper; using TupleColumns = std::vector; TupleColumns columns; @@ -30,7 +30,7 @@ public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. */ - using Base = COWPtrHelper; + using Base = COWHelper; static Ptr create(const Columns & columns); static Ptr create(const TupleColumns & columns); static Ptr create(Columns && arg) { return create(arg); } diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index 5882f9109b2..11344a23a1f 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -25,9 +25,9 @@ namespace ErrorCodes } template -class ColumnUnique final : public COWPtrHelper> +class ColumnUnique final : public COWHelper> { - friend class COWPtrHelper>; + friend class COWHelper>; private: explicit ColumnUnique(MutableColumnPtr && holder, bool is_nullable); diff --git a/dbms/src/Columns/ColumnVector.h b/dbms/src/Columns/ColumnVector.h index 43f6b0a3d52..15773637c1a 100644 --- a/dbms/src/Columns/ColumnVector.h +++ b/dbms/src/Columns/ColumnVector.h @@ -90,13 +90,13 @@ template <> struct CompareHelper : public FloatCompareHelper { /** A template for columns that use a simple array to store. */ template -class ColumnVector final : public COWPtrHelper> +class ColumnVector final : public COWHelper> { static_assert(!IsDecimalNumber); private: using Self = ColumnVector; - friend class COWPtrHelper; + friend class COWHelper; struct less; struct greater; diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index c9afd7c1bfe..74a1302d094 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include @@ -24,13 +24,13 @@ class Arena; class ColumnGathererStream; /// Declares interface to store columns in memory. -class IColumn : public COWPtr +class IColumn : public COW { private: - friend class COWPtr; + friend class COW; /// Creates the same column with the same data. - /// This is internal method to use from COWPtr. + /// This is internal method to use from COW. /// It performs shallow copy with copy-ctor and not useful from outside. /// If you want to copy column for modification, look at 'mutate' method. virtual MutablePtr clone() const = 0; diff --git a/dbms/src/Common/COWPtr.h b/dbms/src/Common/COW.h similarity index 90% rename from dbms/src/Common/COWPtr.h rename to dbms/src/Common/COW.h index 88b4f0b7740..b701c95db97 100644 --- a/dbms/src/Common/COWPtr.h +++ b/dbms/src/Common/COW.h @@ -10,10 +10,10 @@ * * Usage: - class Column : public COWPtr + class Column : public COW { private: - friend class COWPtr; + friend class COW; /// Leave all constructors in private section. They will be avaliable through 'create' method. Column(); @@ -63,7 +63,7 @@ * Actually it is, if your values are small or if copying is done implicitly. * This is the case for string implementations. * - * In contrast, COWPtr is intended for the cases when you need to share states of large objects, + * In contrast, COW is intended for the cases when you need to share states of large objects, * (when you usually will use std::shared_ptr) but you also want precise control over modification * of this shared state. * @@ -73,7 +73,7 @@ * to use std::unique_ptr for it somehow. */ template -class COWPtr : public boost::intrusive_ref_counter +class COW : public boost::intrusive_ref_counter { private: Derived * derived() { return static_cast(this); } @@ -96,8 +96,8 @@ protected: private: using Base = IntrusivePtr; - template friend class COWPtr; - template friend class COWPtrHelper; + template friend class COW; + template friend class COWHelper; explicit mutable_ptr(T * ptr) : Base(ptr) {} @@ -115,7 +115,7 @@ protected: mutable_ptr() = default; - mutable_ptr(const std::nullptr_t *) {} + mutable_ptr(std::nullptr_t) {} }; public: @@ -128,8 +128,8 @@ protected: private: using Base = IntrusivePtr; - template friend class COWPtr; - template friend class COWPtrHelper; + template friend class COW; + template friend class COWHelper; explicit immutable_ptr(const T * ptr) : Base(ptr) {} @@ -159,7 +159,7 @@ protected: immutable_ptr() = default; - immutable_ptr(const std::nullptr_t *) {} + immutable_ptr(std::nullptr_t) {} }; public: @@ -192,7 +192,7 @@ public: MutablePtr assumeMutable() const { - return const_cast(this)->getPtr(); + return const_cast(this)->getPtr(); } Derived & assumeMutableRef() const @@ -244,7 +244,7 @@ public: * * NOTE: * If you override 'mutate' method in inherited classes, don't forget to make it virtual in base class or to make it call a virtual method. - * (COWPtr itself doesn't force any methods to be virtual). + * (COW itself doesn't force any methods to be virtual). * * See example in "cow_compositions.cpp". */ @@ -255,22 +255,22 @@ public: /** Helper class to support inheritance. * Example: * - * class IColumn : public COWPtr + * class IColumn : public COW * { - * friend class COWPtr; + * friend class COW; * virtual MutablePtr clone() const = 0; * virtual ~IColumn() {} * }; * - * class ConcreteColumn : public COWPtrHelper + * class ConcreteColumn : public COWHelper * { - * friend class COWPtrHelper; + * friend class COWHelper; * }; * * Here is complete inheritance diagram: * * ConcreteColumn - * COWPtrHelper + * COWHelper * IColumn * CowPtr * boost::intrusive_ref_counter @@ -278,7 +278,7 @@ public: * See example in "cow_columns.cpp". */ template -class COWPtrHelper : public Base +class COWHelper : public Base { private: Derived * derived() { return static_cast(this); } diff --git a/dbms/src/Common/tests/cow_columns.cpp b/dbms/src/Common/tests/cow_columns.cpp index cf48c159a96..dad2ba13de5 100644 --- a/dbms/src/Common/tests/cow_columns.cpp +++ b/dbms/src/Common/tests/cow_columns.cpp @@ -1,11 +1,11 @@ -#include +#include #include -class IColumn : public COWPtr +class IColumn : public COW { private: - friend class COWPtr; + friend class COW; virtual MutablePtr clone() const = 0; public: @@ -22,10 +22,10 @@ public: using ColumnPtr = IColumn::Ptr; using MutableColumnPtr = IColumn::MutablePtr; -class ConcreteColumn : public COWPtrHelper +class ConcreteColumn : public COWHelper { private: - friend class COWPtrHelper; + friend class COWHelper; int data; ConcreteColumn(int data) : data(data) {} diff --git a/dbms/src/Common/tests/cow_compositions.cpp b/dbms/src/Common/tests/cow_compositions.cpp index 76b8a2987ab..a48624d7d64 100644 --- a/dbms/src/Common/tests/cow_compositions.cpp +++ b/dbms/src/Common/tests/cow_compositions.cpp @@ -1,11 +1,11 @@ -#include +#include #include -class IColumn : public COWPtr +class IColumn : public COW { private: - friend class COWPtr; + friend class COW; virtual MutablePtr clone() const = 0; virtual MutablePtr deepMutate() const { return shallowMutate(); } @@ -24,10 +24,10 @@ public: using ColumnPtr = IColumn::Ptr; using MutableColumnPtr = IColumn::MutablePtr; -class ConcreteColumn : public COWPtrHelper +class ConcreteColumn : public COWHelper { private: - friend class COWPtrHelper; + friend class COWHelper; int data; ConcreteColumn(int data) : data(data) {} @@ -38,10 +38,10 @@ public: void set(int value) override { data = value; } }; -class ColumnComposition : public COWPtrHelper +class ColumnComposition : public COWHelper { private: - friend class COWPtrHelper; + friend class COWHelper; ConcreteColumn::WrappedPtr wrapped; diff --git a/dbms/src/Core/iostream_debug_helpers.cpp b/dbms/src/Core/iostream_debug_helpers.cpp index 98a9775f15d..1f36e081f35 100644 --- a/dbms/src/Core/iostream_debug_helpers.cpp +++ b/dbms/src/Core/iostream_debug_helpers.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index aa253fbdc08..60124cd3d5d 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include @@ -17,8 +17,8 @@ class IDataType; struct FormatSettings; class IColumn; -using ColumnPtr = COWPtr::Ptr; -using MutableColumnPtr = COWPtr::MutablePtr; +using ColumnPtr = COW::Ptr; +using MutableColumnPtr = COW::MutablePtr; using DataTypePtr = std::shared_ptr; using DataTypes = std::vector; From e914d0d67ae86fb2eb1536f670785ada2f58b836 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Sat, 20 Apr 2019 15:47:16 +0300 Subject: [PATCH 14/37] Just update LZ4 --- dbms/src/Compression/CompressionCodecLZ4.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/dbms/src/Compression/CompressionCodecLZ4.cpp b/dbms/src/Compression/CompressionCodecLZ4.cpp index 5a3085d54f9..08553e0920c 100644 --- a/dbms/src/Compression/CompressionCodecLZ4.cpp +++ b/dbms/src/Compression/CompressionCodecLZ4.cpp @@ -20,7 +20,6 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_COMPRESS; -extern const int CANNOT_DECOMPRESS; extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; extern const int ILLEGAL_CODEC_PARAMETER; } @@ -48,9 +47,7 @@ UInt32 CompressionCodecLZ4::doCompressData(const char * source, UInt32 source_si void CompressionCodecLZ4::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { - if (LZ4_decompress_safe(source, dest, source_size, uncompressed_size) < 0) - throw Exception("Cannot LZ4_decompress_safe", ErrorCodes::CANNOT_DECOMPRESS); - // LZ4::decompress(source, dest, source_size, uncompressed_size, lz4_stat); + LZ4::decompress(source, dest, source_size, uncompressed_size, lz4_stat); } void registerCodecLZ4(CompressionCodecFactory & factory) From e62999d8b91f9e3f92138d0f7a02ec268e625b34 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 20 Apr 2019 23:25:19 +0300 Subject: [PATCH 15/37] Fixed typo in comment. --- dbms/src/Common/COW.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/COW.h b/dbms/src/Common/COW.h index b701c95db97..d8152af8356 100644 --- a/dbms/src/Common/COW.h +++ b/dbms/src/Common/COW.h @@ -23,7 +23,7 @@ public: /// Correctly use const qualifiers in your interface. - virtual ~IColumn() {} + virtual ~Column() {} }; * It will provide 'create' and 'mutate' methods. From fdc8b397c86583079524b0b2ba84d24e2da14ac9 Mon Sep 17 00:00:00 2001 From: never lee Date: Mon, 22 Apr 2019 16:27:13 +0800 Subject: [PATCH 16/37] Zhdocs/replication md (#5068) --- .../operations/table_engines/replication.md | 2 +- docs/zh/operations/table_engines/mergetree.md | 30 ++-- .../operations/table_engines/replication.md | 155 +++++++++--------- 3 files changed, 95 insertions(+), 92 deletions(-) diff --git a/docs/en/operations/table_engines/replication.md b/docs/en/operations/table_engines/replication.md index 0b4ccc496fa..d6900ca2f32 100644 --- a/docs/en/operations/table_engines/replication.md +++ b/docs/en/operations/table_engines/replication.md @@ -55,7 +55,7 @@ For very large clusters, you can use different ZooKeeper clusters for different Replication is asynchronous and multi-master. `INSERT` queries (as well as `ALTER`) can be sent to any available server. Data is inserted on the server where the query is run, and then it is copied to the other servers. Because it is asynchronous, recently inserted data appears on the other replicas with some latency. If part of the replicas are not available, the data is written when they become available. If a replica is available, the latency is the amount of time it takes to transfer the block of compressed data over the network. -By default, an INSERT query waits for confirmation of writing the data from only one replica. If the data was successfully written to only one replica and the server with this replica ceases to exist, the stored data will be lost. Tp enable getting confirmation of data writes from multiple replicas, use the `insert_quorum` option. +By default, an INSERT query waits for confirmation of writing the data from only one replica. If the data was successfully written to only one replica and the server with this replica ceases to exist, the stored data will be lost. To enable getting confirmation of data writes from multiple replicas, use the `insert_quorum` option. Each block of data is written atomically. The INSERT query is divided into blocks up to `max_insert_block_size = 1048576` rows. In other words, if the `INSERT` query has less than 1048576 rows, it is made atomically. diff --git a/docs/zh/operations/table_engines/mergetree.md b/docs/zh/operations/table_engines/mergetree.md index 953f5128337..c8febd0a21b 100644 --- a/docs/zh/operations/table_engines/mergetree.md +++ b/docs/zh/operations/table_engines/mergetree.md @@ -1,8 +1,8 @@ # MergeTree {#table_engines-mergetree} -Clickhouse 中最强大的表引擎当属 `MergeTree` (合并树)引擎及该家族(`*MergeTree`)中的其他引擎。 +Clickhouse 中最强大的表引擎当属 `MergeTree` (合并树)引擎及该系列(`*MergeTree`)中的其他引擎。 -`MergeTree` 引擎家族的基本理念如下。当你有巨量数据要插入到表中,你要高效地一批批写入数据分片,并希望这些数据分片在后台按照一定规则合并。相比在插入时不断修改(重写)数据进存储,这种策略会高效很多。 +`MergeTree` 引擎系列的基本理念如下。当你有巨量数据要插入到表中,你要高效地一批批写入数据片段,并希望这些数据片段在后台按照一定规则合并。相比在插入时不断修改(重写)数据进存储,这种策略会高效很多。 主要特点: @@ -16,14 +16,14 @@ Clickhouse 中最强大的表引擎当属 `MergeTree` (合并树)引擎及 - 支持数据副本。 - `ReplicatedMergeTree` 家族的表便是用于此。更多信息,请参阅 [数据副本](replication.md) 一节。 + `ReplicatedMergeTree` 系列的表便是用于此。更多信息,请参阅 [数据副本](replication.md) 一节。 - 支持数据采样。 需要的话,你可以给表设置一个采样方法。 !!! 注意 - [Merge](merge.md) 引擎并不属于 `*MergeTree` 家族。 + [Merge](merge.md) 引擎并不属于 `*MergeTree` 系列。 ## 建表 {#table_engine-mergetree-creating-a-table} @@ -70,8 +70,8 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] - `SETTINGS` — 影响 `MergeTree` 性能的额外参数: - `index_granularity` — 索引粒度。即索引中相邻『标记』间的数据行数。默认值,8192 。该列表中所有可用的参数可以从这里查看 [MergeTreeSettings.h](https://github.com/yandex/ClickHouse/blob/master/dbms/src/Storages/MergeTree/MergeTreeSettings.h) 。 - - `use_minimalistic_part_header_in_zookeeper` — 数据分片头在 ZooKeeper 中的存储方式。如果设置了 `use_minimalistic_part_header_in_zookeeper=1` ,ZooKeeper 会存储更少的数据。更多信息参考『服务配置参数』这章中的 [设置描述](../server_settings/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) 。 - - `min_merge_bytes_to_use_direct_io` — 使用直接 I/O 来操作磁盘的合并操作时要求的最小数据量。合并数据分片时,ClickHouse 会计算要被合并的所有数据的总存储空间。如果大小超过了 `min_merge_bytes_to_use_direct_io` 设置的字节数,则 ClickHouse 将使用直接 I/O 接口(`O_DIRECT` 选项)对磁盘读写。如果设置 `min_merge_bytes_to_use_direct_io = 0` ,则会禁用直接 I/O。默认值:`10 * 1024 * 1024 * 1024` 字节。 + - `use_minimalistic_part_header_in_zookeeper` — 数据片段头在 ZooKeeper 中的存储方式。如果设置了 `use_minimalistic_part_header_in_zookeeper=1` ,ZooKeeper 会存储更少的数据。更多信息参考『服务配置参数』这章中的 [设置描述](../server_settings/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) 。 + - `min_merge_bytes_to_use_direct_io` — 使用直接 I/O 来操作磁盘的合并操作时要求的最小数据量。合并数据片段时,ClickHouse 会计算要被合并的所有数据的总存储空间。如果大小超过了 `min_merge_bytes_to_use_direct_io` 设置的字节数,则 ClickHouse 将使用直接 I/O 接口(`O_DIRECT` 选项)对磁盘读写。如果设置 `min_merge_bytes_to_use_direct_io = 0` ,则会禁用直接 I/O。默认值:`10 * 1024 * 1024 * 1024` 字节。 **示例配置** @@ -117,13 +117,13 @@ MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID) ## 数据存储 -表由按主键排序的数据 *分片* 组成。 +表由按主键排序的数据 *片段* 组成。 -当数据被插入到表中时,会分成数据分片并按主键的字典序排序。例如,主键是 `(CounterID, Date)` 时,分片中数据按 `CounterID` 排序,具有相同 `CounterID` 的部分按 `Date` 排序。 +当数据被插入到表中时,会分成数据片段并按主键的字典序排序。例如,主键是 `(CounterID, Date)` 时,片段中数据按 `CounterID` 排序,具有相同 `CounterID` 的部分按 `Date` 排序。 -不同分区的数据会被分成不同的分片,ClickHouse 在后台合并数据分片以便更高效存储。不会合并来自不同分区的数据分片。这个合并机制并不保证相同主键的所有行都会合并到同一个数据分片中。 +不同分区的数据会被分成不同的片段,ClickHouse 在后台合并数据片段以便更高效存储。不会合并来自不同分区的数据片段。这个合并机制并不保证相同主键的所有行都会合并到同一个数据片段中。 -ClickHouse 会为每个数据分片创建一个索引文件,索引文件包含每个索引行(『标记』)的主键值。索引行号定义为 `n * index_granularity` 。最大的 `n` 等于总行数除以 `index_granularity` 的值的整数部分。对于每列,跟主键相同的索引行处也会写入『标记』。这些『标记』让你可以直接找到数据所在的列。 +ClickHouse 会为每个数据片段创建一个索引文件,索引文件包含每个索引行(『标记』)的主键值。索引行号定义为 `n * index_granularity` 。最大的 `n` 等于总行数除以 `index_granularity` 的值的整数部分。对于每列,跟主键相同的索引行处也会写入『标记』。这些『标记』让你可以直接找到数据所在的列。 你可以只用一单一大表并不断地一块块往里面加入数据 – `MergeTree` 引擎的就是为了这样的场景。 @@ -166,7 +166,7 @@ ClickHouse 不要求主键惟一。所以,你可以插入多条具有相同主 - 改善数据压缩。 - ClickHouse 以主键排序分片数据,所以,数据的一致性越高,压缩越好。 + ClickHouse 以主键排序片段数据,所以,数据的一致性越高,压缩越好。 - [CollapsingMergeTree](collapsingmergetree.md#table_engine-collapsingmergetree) 和 [SummingMergeTree](summingmergetree.md) 引擎里,数据合并时,会有额外的处理逻辑。 @@ -177,7 +177,7 @@ ClickHouse 不要求主键惟一。所以,你可以插入多条具有相同主 ### 选择跟排序键不一样主键 -指定一个跟排序键(用于排序数据分片中行的表达式) +指定一个跟排序键(用于排序数据片段中行的表达式) 不一样的主键(用于计算写到索引文件的每个标记值的表达式)是可以的。 这种情况下,主键表达式元组必须是排序键表达式元组的一个前缀。 @@ -192,7 +192,7 @@ ClickHouse 不要求主键惟一。所以,你可以插入多条具有相同主 这种情况下,主键中仅预留少量列保证高效范围扫描, 剩下的维度列放到排序键元组里。这样是合理的。 -[排序键的修改](../../query_language/alter.md) 是轻量级的操作,因为一个新列同时被加入到表里和排序键后时,已存在的数据分片并不需要修改。由于旧的排序键是新排序键的前缀,并且刚刚添加的列中没有数据,因此在表修改时的数据对于新旧的排序键来说都是有序的。 +[排序键的修改](../../query_language/alter.md) 是轻量级的操作,因为一个新列同时被加入到表里和排序键后时,已存在的数据片段并不需要修改。由于旧的排序键是新排序键的前缀,并且刚刚添加的列中没有数据,因此在表修改时的数据对于新旧的排序键来说都是有序的。 ### 索引和分区在查询中的应用 @@ -238,7 +238,7 @@ SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' INDEX index_name expr TYPE type(...) GRANULARITY granularity_value ``` -`*MergeTree` 家族的表都能指定跳数索引。 +`*MergeTree` 系列的表都能指定跳数索引。 这些索引是由数据块按粒度分割后的每部分在指定表达式上汇总信息 `granularity_value` 组成(粒度大小用表引擎里 `index_granularity` 的指定)。 这些汇总信息有助于用 `where` 语句跳过大片不满足的数据,从而减少 `SELECT` 查询从磁盘读取的数据量, @@ -292,7 +292,7 @@ INDEX sample_index3 (lower(str), str) TYPE ngrambf_v1(3, 256, 2, 0) GRANULARITY ## 并发数据访问 -应对表的并发访问,我们使用多版本机制。换言之,当同时读和更新表时,数据从当前查询到的一组分片中读取。没有冗长的的锁。插入不会阻碍读取。 +应对表的并发访问,我们使用多版本机制。换言之,当同时读和更新表时,数据从当前查询到的一组片段中读取。没有冗长的的锁。插入不会阻碍读取。 对表的读操作是自动并行的。 diff --git a/docs/zh/operations/table_engines/replication.md b/docs/zh/operations/table_engines/replication.md index 0564408ca76..76a465de978 100644 --- a/docs/zh/operations/table_engines/replication.md +++ b/docs/zh/operations/table_engines/replication.md @@ -1,27 +1,28 @@ -# Data Replication {#table_engines-replication} +# 数据副本 {#table_engines-replication} -Replication is only supported for tables in the MergeTree family: +只有 MergeTree 系列里的表可支持副本: - ReplicatedMergeTree - ReplicatedSummingMergeTree - ReplicatedReplacingMergeTree - ReplicatedAggregatingMergeTree - ReplicatedCollapsingMergeTree +- ReplicatedVersionedCollapsingMergeTree - ReplicatedGraphiteMergeTree -Replication works at the level of an individual table, not the entire server. A server can store both replicated and non-replicated tables at the same time. +副本是表级别的,不是整个服务器级的。所以,服务器里可以同时有复制表和非复制表。 -Replication does not depend on sharding. Each shard has its own independent replication. +副本不依赖分片。每个分片有它自己的独立副本。 -Compressed data for `INSERT` and `ALTER` queries is replicated (for more information, see the documentation for [ALTER](../../query_language/alter.md#query_language_queries_alter)). +对于 `INSERT` 和 `ALTER` 语句操作数据的会在压缩的情况下被复制(更多信息,看 [ALTER](../../query_language/alter.md#query_language_queries_alter) )。 -`CREATE`, `DROP`, `ATTACH`, `DETACH` and `RENAME` queries are executed on a single server and are not replicated: +而 `CREATE`,`DROP`,`ATTACH`,`DETACH` 和 `RENAME` 语句只会在单个服务器上执行,不会被复制。 -- `The CREATE TABLE` query creates a new replicatable table on the server where the query is run. If this table already exists on other servers, it adds a new replica. -- `The DROP TABLE` query deletes the replica located on the server where the query is run. -- `The RENAME` query renames the table on one of the replicas. In other words, replicated tables can have different names on different replicas. +- `The CREATE TABLE` 在运行此语句的服务器上创建一个新的可复制表。如果此表已存在其他服务器上,则给该表添加新副本。 +- `The DROP TABLE` 删除运行此查询的服务器上的副本。 +- `The RENAME` 重命名一个副本。换句话说,可复制表不同的副本可以有不同的名称。 -To use replication, set the addresses of the ZooKeeper cluster in the config file. Example: +要使用副本,需在配置文件中设置 ZooKeeper 集群的地址。例如: ```xml @@ -40,43 +41,44 @@ To use replication, set the addresses of the ZooKeeper cluster in the config fil ``` -Use ZooKeeper version 3.4.5 or later. +需要 ZooKeeper 3.4.5 或更高版本。 -You can specify any existing ZooKeeper cluster and the system will use a directory on it for its own data (the directory is specified when creating a replicatable table). +你可以配置任何现有的 ZooKeeper 集群,系统会使用里面的目录来存取元数据(该目录在创建可复制表时指定)。 -If ZooKeeper isn't set in the config file, you can't create replicated tables, and any existing replicated tables will be read-only. +如果配置文件中没有设置 ZooKeeper ,则无法创建复制表,并且任何现有的复制表都将变为只读。 -ZooKeeper is not used in `SELECT` queries because replication does not affect the performance of `SELECT` and queries run just as fast as they do for non-replicated tables. When querying distributed replicated tables, ClickHouse behavior is controlled by the settings [max_replica_delay_for_distributed_queries](../settings/settings.md#settings-max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](../settings/settings.md). + `SELECT` 查询并不需要借助 ZooKeeper ,复本并不影响 `SELECT` 的性能,查询复制表与非复制表速度是一样的。查询分布式表时,ClickHouse的处理方式可通过设置 [max_replica_delay_for_distributed_queries](../settings/settings.md#settings-max_replica_delay_for_distributed_queries) 和 [fallback_to_stale_replicas_for_distributed_queries](../settings/settings.md) 修改。 -For each `INSERT` query, approximately ten entries are added to ZooKeeper through several transactions. (To be more precise, this is for each inserted block of data; an INSERT query contains one block or one block per `max_insert_block_size = 1048576` rows.) This leads to slightly longer latencies for `INSERT` compared to non-replicated tables. But if you follow the recommendations to insert data in batches of no more than one `INSERT` per second, it doesn't create any problems. The entire ClickHouse cluster used for coordinating one ZooKeeper cluster has a total of several hundred `INSERTs` per second. The throughput on data inserts (the number of rows per second) is just as high as for non-replicated data. +对于每个 `INSERT` 语句,会通过几个事务将十来个记录添加到 ZooKeeper。(确切地说,这是针对每个插入的数据块; 每个 INSERT 语句的每 `max_insert_block_size = 1048576` 行和最后剩余的都各算作一个块。)相比非复制表,写 zk 会导致 `INSERT` 的延迟略长一些。但只要你按照建议每秒不超过一个 `INSERT` 地批量插入数据,不会有任何问题。一个 ZooKeeper 集群能给整个 ClickHouse 集群支撑协调每秒几百个 `INSERT`。数据插入的吞吐量(每秒的行数)可以跟不用复制的数据一样高。 -For very large clusters, you can use different ZooKeeper clusters for different shards. However, this hasn't proven necessary on the Yandex.Metrica cluster (approximately 300 servers). +对于非常大的集群,你可以把不同的 ZooKeeper 集群用于不同的分片。然而,即使 Yandex.Metrica 集群(大约300台服务器)也证明还不需要这么做。 -Replication is asynchronous and multi-master. `INSERT` queries (as well as `ALTER`) can be sent to any available server. Data is inserted on the server where the query is run, and then it is copied to the other servers. Because it is asynchronous, recently inserted data appears on the other replicas with some latency. If part of the replicas are not available, the data is written when they become available. If a replica is available, the latency is the amount of time it takes to transfer the block of compressed data over the network. +复制是多主异步。 `INSERT` 语句(以及 `ALTER` )可以发给任意可用的服务器。数据会先插入到执行该语句的服务器上,然后被复制到其他服务器。由于它是异步的,在其他副本上最近插入的数据会有一些延迟。如果部分副本不可用,则数据在其可用时再写入。副本可用的情况下,则延迟时长是通过网络传输压缩数据块所需的时间。 -By default, an INSERT query waits for confirmation of writing the data from only one replica. If the data was successfully written to only one replica and the server with this replica ceases to exist, the stored data will be lost. Tp enable getting confirmation of data writes from multiple replicas, use the `insert_quorum` option. +默认情况下,INSERT 语句仅等待一个副本写入成功后返回。如果数据只成功写入一个副本后该副本所在的服务器不再存在,则存储的数据会丢失。要启用数据写入多个副本才确认返回,使用 `insert_quorum` 选项。 -Each block of data is written atomically. The INSERT query is divided into blocks up to `max_insert_block_size = 1048576` rows. In other words, if the `INSERT` query has less than 1048576 rows, it is made atomically. +单个数据块写入是原子的。 INSERT 的数据按每块最多 `max_insert_block_size = 1048576` 行进行分块,换句话说,如果 `INSERT` 插入的行少于 1048576,则该 INSERT 是原子的。 -Data blocks are deduplicated. For multiple writes of the same data block (data blocks of the same size containing the same rows in the same order), the block is only written once. The reason for this is in case of network failures when the client application doesn't know if the data was written to the DB, so the `INSERT` query can simply be repeated. It doesn't matter which replica INSERTs were sent to with identical data. `INSERTs` are idempotent. Deduplication parameters are controlled by [merge_tree](../server_settings/settings.md) server settings. +数据块会去重。对于被多次写的相同数据块(大小相同且具有相同顺序的相同行的数据块),该块仅会写入一次。这样设计的原因是万一在网络故障时客户端应用程序不知道数据是否成功写入DB,此时可以简单地重复 `INSERT` 。把相同的数据发送给多个副本 INSERT 并不会有问题。因为这些 `INSERT` 是完全相同的(会被去重)。去重参数参看服务器设置 [merge_tree](../server_settings/settings.md) 。(注意:Replicated\*MergeTree 才会去重,不需要 zookeeper 的不带 MergeTree 不会去重) -During replication, only the source data to insert is transferred over the network. Further data transformation (merging) is coordinated and performed on all the replicas in the same way. This minimizes network usage, which means that replication works well when replicas reside in different datacenters. (Note that duplicating data in different datacenters is the main goal of replication.) +在复制期间,只有要插入的源数据通过网络传输。进一步的数据转换(合并)会在所有副本上以相同的方式进行处理执行。这样可以最大限度地减少网络使用,这意味着即使副本在不同的数据中心,数据同步也能工作良好。(能在不同数据中心中的同步数据是副本机制的主要目标。) -You can have any number of replicas of the same data. Yandex.Metrica uses double replication in production. Each server uses RAID-5 or RAID-6, and RAID-10 in some cases. This is a relatively reliable and convenient solution. +你可以给数据做任意多的副本。Yandex.Metrica 在生产中使用双副本。某一些情况下,给每台服务器都使用 RAID-5 或 RAID-6 和 RAID-10。是一种相对可靠和方便的解决方案。 -The system monitors data synchronicity on replicas and is able to recover after a failure. Failover is automatic (for small differences in data) or semi-automatic (when data differs too much, which may indicate a configuration error). +系统会监视副本数据同步情况,并能在发生故障后恢复。故障转移是自动的(对于小的数据差异)或半自动的(当数据差异很大时,这可能意味是有配置错误)。 -## Creating Replicated Tables +## 创建复制表 {#creating-replicated-tables} -The `Replicated` prefix is added to the table engine name. For example:`ReplicatedMergeTree`. -**Replicated\*MergeTree parameters** +在表引擎名称上加上 `Replicated` 前缀。例如:`ReplicatedMergeTree`。 -- `zoo_path` — The path to the table in ZooKeeper. -- `replica_name` — The replica name in ZooKeeper. +**Replicated\*MergeTree 参数** -Example: +- `zoo_path` — ZooKeeper 中该表的路径。 +- `replica_name` — ZooKeeper 中的该表的副本名称。 + +示例: ```sql CREATE TABLE table_name @@ -90,7 +92,7 @@ ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) ``` -Example in deprecated syntax: +已弃用的建表语法示例: ```sql CREATE TABLE table_name @@ -101,7 +103,7 @@ CREATE TABLE table_name ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/hits', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192) ``` -As the example shows, these parameters can contain substitutions in curly brackets. The substituted values are taken from the 'macros' section of the configuration file. Example: +如上例所示,这些参数可以包含宏替换的占位符,即大括号的部分。它们会被替换为配置文件里 'macros' 那部分配置的值。示例: ```xml @@ -111,92 +113,93 @@ As the example shows, these parameters can contain substitutions in curly bracke ``` -The path to the table in ZooKeeper should be unique for each replicated table. Tables on different shards should have different paths. -In this case, the path consists of the following parts: +“ZooKeeper 中该表的路径”对每个可复制表都要是唯一的。不同分片上的表要有不同的路径。 +这种情况下,路径包含下面这些部分: -`/clickhouse/tables/` is the common prefix. We recommend using exactly this one. +`/clickhouse/tables/` 是公共前缀,我们推荐使用这个。 -`{layer}-{shard}` is the shard identifier. In this example it consists of two parts, since the Yandex.Metrica cluster uses bi-level sharding. For most tasks, you can leave just the {shard} substitution, which will be expanded to the shard identifier. +`{layer}-{shard}` 是分片标识部分。在此示例中,由于 Yandex.Metrica 集群使用了两级分片,所以它是由两部分组成的。但对于大多数情况来说,你只需保留 {shard} 占位符即可,它会替换展开为分片标识。 -`hits` is the name of the node for the table in ZooKeeper. It is a good idea to make it the same as the table name. It is defined explicitly, because in contrast to the table name, it doesn't change after a RENAME query. +`hits` 是该表在 ZooKeeper 中的名称。使其与 ClickHouse 中的表名相同比较好。 这里它被明确定义,跟 ClickHouse 表名不一样,它并不会被 RENAME 语句修改。 -The replica name identifies different replicas of the same table. You can use the server name for this, as in the example. The name only needs to be unique within each shard. +副本名称用于标识同一个表分片的不同副本。你可以使用服务器名称,如上例所示。同个分片中不同副本的副本名称要唯一。 -You can define the parameters explicitly instead of using substitutions. This might be convenient for testing and for configuring small clusters. However, you can't use distributed DDL queries (`ON CLUSTER`) in this case. +你也可以显式指定这些参数,而不是使用宏替换。对于测试和配置小型集群这可能会很方便。但是,这种情况下,则不能使用分布式 DDL 语句(`ON CLUSTER`)。 -When working with large clusters, we recommend using substitutions because they reduce the probability of error. +使用大型集群时,我们建议使用宏替换,因为它可以降低出错的可能性。 -Run the `CREATE TABLE` query on each replica. This query creates a new replicated table, or adds a new replica to an existing one. +在每个副本服务器上运行 `CREATE TABLE` 查询。将创建新的复制表,或给现有表添加新副本。 -If you add a new replica after the table already contains some data on other replicas, the data will be copied from the other replicas to the new one after running the query. In other words, the new replica syncs itself with the others. +如果其他副本上已包含了某些数据,在表上添加新副本,则在运行语句后,数据会从其他副本复制到新副本。换句话说,新副本会与其他副本同步。 -To delete a replica, run `DROP TABLE`. However, only one replica is deleted – the one that resides on the server where you run the query. +要删除副本,使用 `DROP TABLE`。但它只删除那个 – 位于运行该语句的服务器上的副本。 -## Recovery After Failures +## 故障恢复 -If ZooKeeper is unavailable when a server starts, replicated tables switch to read-only mode. The system periodically attempts to connect to ZooKeeper. +如果服务器启动时 ZooKeeper 不可用,则复制表会切换为只读模式。系统会定期尝试去连接 ZooKeeper。 -If ZooKeeper is unavailable during an `INSERT`, or an error occurs when interacting with ZooKeeper, an exception is thrown. +如果在 `INSERT` 期间 ZooKeeper 不可用,或者在与 ZooKeeper 交互时发生错误,则抛出异常。 -After connecting to ZooKeeper, the system checks whether the set of data in the local file system matches the expected set of data (ZooKeeper stores this information). If there are minor inconsistencies, the system resolves them by syncing data with the replicas. +连接到 ZooKeeper 后,系统会检查本地文件系统中的数据集是否与预期的数据集( ZooKeeper 存储此信息)一致。如果存在轻微的不一致,系统会通过与副本同步数据来解决。 -If the system detects broken data parts (with the wrong size of files) or unrecognized parts (parts written to the file system but not recorded in ZooKeeper), it moves them to the 'detached' subdirectory (they are not deleted). Any missing parts are copied from the replicas. +如果系统检测到损坏的数据片段(文件大小错误)或无法识别的片段(写入文件系统但未记录在 ZooKeeper 中的部分),则会把它们移动到 'detached' 子目录(不会删除)。而副本中其他任何缺少的但正常数据片段都会被复制同步。 -Note that ClickHouse does not perform any destructive actions such as automatically deleting a large amount of data. +注意,ClickHouse 不会执行任何破坏性操作,例如自动删除大量数据。 -When the server starts (or establishes a new session with ZooKeeper), it only checks the quantity and sizes of all files. If the file sizes match but bytes have been changed somewhere in the middle, this is not detected immediately, but only when attempting to read the data for a `SELECT` query. The query throws an exception about a non-matching checksum or size of a compressed block. In this case, data parts are added to the verification queue and copied from the replicas if necessary. +当服务器启动(或与 ZooKeeper 建立新会话)时,它只检查所有文件的数量和大小。 如果文件大小一致但中间某处已有字节被修改过,不会立即被检测到,只有在尝试读取 `SELECT` 查询的数据时才会检测到。该查询会引发校验和不匹配或压缩块大小不一致的异常。这种情况下,数据片段会添加到验证队列中,并在必要时从其他副本中复制。 -If the local set of data differs too much from the expected one, a safety mechanism is triggered. The server enters this in the log and refuses to launch. The reason for this is that this case may indicate a configuration error, such as if a replica on a shard was accidentally configured like a replica on a different shard. However, the thresholds for this mechanism are set fairly low, and this situation might occur during normal failure recovery. In this case, data is restored semi-automatically - by "pushing a button". +如果本地数据集与预期数据的差异太大,则会触发安全机制。服务器在日志中记录此内容并拒绝启动。这种情况很可能是配置错误,例如,一个分片上的副本意外配置为别的分片上的副本。然而,此机制的阈值设置得相当低,在正常故障恢复期间可能会出现这种情况。在这种情况下,数据恢复则是半自动模式,通过用户主动操作触发。 -To start recovery, create the node `/path_to_table/replica_name/flags/force_restore_data` in ZooKeeper with any content, or run the command to restore all replicated tables: +要触发启动恢复,可在 ZooKeeper 中创建节点 `/path_to_table/replica_name/flags/force_restore_data`,节点值可以是任何内容,或运行命令来恢复所有的可复制表: ```bash sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data ``` -Then restart the server. On start, the server deletes these flags and starts recovery. +然后重启服务器。启动时,服务器会删除这些标志并开始恢复。 -## Recovery After Complete Data Loss +## 在数据完全丢失后的恢复 -If all data and metadata disappeared from one of the servers, follow these steps for recovery: +如果其中一个服务器的所有数据和元数据都消失了,请按照以下步骤进行恢复: -1. Install ClickHouse on the server. Define substitutions correctly in the config file that contains the shard identifier and replicas, if you use them. -2. If you had unreplicated tables that must be manually duplicated on the servers, copy their data from a replica (in the directory `/var/lib/clickhouse/data/db_name/table_name/`). -3. Copy table definitions located in `/var/lib/clickhouse/metadata/` from a replica. If a shard or replica identifier is defined explicitly in the table definitions, correct it so that it corresponds to this replica. (Alternatively, start the server and make all the `ATTACH TABLE` queries that should have been in the .sql files in `/var/lib/clickhouse/metadata/`.) -4. To start recovery, create the ZooKeeper node `/path_to_table/replica_name/flags/force_restore_data` with any content, or run the command to restore all replicated tables: `sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data` +1. 在服务器上安装 ClickHouse。在包含分片标识符和副本的配置文件中正确定义宏配置,如果有用到的话, +2. 如果服务器上有非复制表则必须手动复制,可以从副本服务器上(在 `/var/lib/clickhouse/data/db_name/table_name/` 目录中)复制它们的数据。 +3. 从副本服务器上中复制位于 `/var/lib/clickhouse/metadata/` 中的表定义信息。如果在表定义信息中显式指定了分片或副本标识符,请更正它以使其对应于该副本。(另外,启动服务器,然后会在 `/var/lib/clickhouse/metadata/` 中的.sql文件中生成所有的 `ATTACH TABLE` 语句。) +4.要开始恢复,ZooKeeper 中创建节点 `/path_to_table/replica_name/flags/force_restore_data`,节点内容不限,或运行命令来恢复所有复制的表:`sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data` -Then start the server (restart, if it is already running). Data will be downloaded from replicas. +然后启动服务器(如果它已运行则重启)。数据会从副本中下载。 -An alternative recovery option is to delete information about the lost replica from ZooKeeper (`/path_to_table/replica_name`), then create the replica again as described in "[Creating replicated tables](#creating-replicated-tables)". +另一种恢复方式是从 ZooKeeper(`/path_to_table/replica_name`)中删除有数据丢的副本的所有元信息,然后再按照“[创建可复制表](#creating-replicated-tables)”中的描述重新创建副本。 -There is no restriction on network bandwidth during recovery. Keep this in mind if you are restoring many replicas at once. +恢复期间的网络带宽没有限制。特别注意这一点,尤其是要一次恢复很多副本。 -## Converting from MergeTree to ReplicatedMergeTree +## MergeTree 转换为 ReplicatedMergeTree -We use the term `MergeTree` to refer to all table engines in the ` MergeTree family`, the same as for `ReplicatedMergeTree`. +我们使用 `MergeTree` 来表示 `MergeTree系列` 中的所有表引擎,`ReplicatedMergeTree` 同理。 -If you had a `MergeTree` table that was manually replicated, you can convert it to a replicatable table. You might need to do this if you have already collected a large amount of data in a `MergeTree` table and now you want to enable replication. +如果你有一个手动同步的 `MergeTree` 表,您可以将其转换为可复制表。如果你已经在 `MergeTree` 表中收集了大量数据,并且现在要启用复制,则可以执行这些操作。 -If the data differs on various replicas, first sync it, or delete this data on all the replicas except one. +如果各个副本上的数据不一致,则首先对其进行同步,或者除保留的一个副本外,删除其他所有副本上的数据。 -Rename the existing MergeTree table, then create a `ReplicatedMergeTree` table with the old name. -Move the data from the old table to the 'detached' subdirectory inside the directory with the new table data (`/var/lib/clickhouse/data/db_name/table_name/`). -Then run `ALTER TABLE ATTACH PARTITION` on one of the replicas to add these data parts to the working set. +重命名现有的 MergeTree 表,然后使用旧名称创建 `ReplicatedMergeTree` 表。 +将数据从旧表移动到新表(`/var/lib/clickhouse/data/db_name/table_name/`)目录内的 'detached' 目录中。 +然后在其中一个副本上运行`ALTER TABLE ATTACH PARTITION`,将这些数据片段添加到工作集中。 -## Converting from ReplicatedMergeTree to MergeTree +## ReplicatedMergeTree 转换为 MergeTree -Create a MergeTree table with a different name. Move all the data from the directory with the `ReplicatedMergeTree` table data to the new table's data directory. Then delete the `ReplicatedMergeTree` table and restart the server. +使用其他名称创建 MergeTree 表。将具有`ReplicatedMergeTree`表数据的目录中的所有数据移动到新表的数据目录中。然后删除`ReplicatedMergeTree`表并重新启动服务器。 +如果你想在不启动服务器的情况下清除 `ReplicatedMergeTree` 表: If you want to get rid of a `ReplicatedMergeTree` table without launching the server: -- Delete the corresponding `.sql` file in the metadata directory (`/var/lib/clickhouse/metadata/`). -- Delete the corresponding path in ZooKeeper (`/path_to_table/replica_name`). +- 删除元数据目录中的相应 `.sql` 文件(`/var/lib/clickhouse/metadata/`)。 +- 删除 ZooKeeper 中的相应路径(`/path_to_table/replica_name`)。 -After this, you can launch the server, create a `MergeTree` table, move the data to its directory, and then restart the server. +之后,你可以启动服务器,创建一个 `MergeTree` 表,将数据移动到其目录,然后重新启动服务器。 -## Recovery When Metadata in The ZooKeeper Cluster is Lost or Damaged +## 当 ZooKeeper 集群中的元数据丢失或损坏时恢复方法 -If the data in ZooKeeper was lost or damaged, you can save data by moving it to an unreplicated table as described above. +如果 ZooKeeper 中的数据丢失或损坏,如上所述,你可以通过将数据转移到非复制表来保存数据。 -[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/replication/) +[来源文章](https://clickhouse.yandex/docs/en/operations/table_engines/replication/) From 19e4352759200949d14ddda33918a54ab7a86a63 Mon Sep 17 00:00:00 2001 From: ogorbacheva Date: Mon, 22 Apr 2019 11:34:25 +0300 Subject: [PATCH 17/37] Doc fix: Actualize `ARRAY JOIN` (#5065) --- docs/en/query_language/select.md | 66 ++--- .../functions/array_functions.md | 2 +- docs/ru/query_language/select.md | 229 +++++++++--------- 3 files changed, 156 insertions(+), 141 deletions(-) diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index 458fa732f81..5e0877430b2 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -156,7 +156,7 @@ Here, a sample of 10% is taken from the second half of the data. ### ARRAY JOIN Clause {#select-array-join-clause} -Allows executing `JOIN` with an array or nested data structure. Allows you to perform `JOIN` both with the external array and with the inner array in the table. The intent is similar to the [arrayJoin](functions/array_functions.md#array_functions-join) function, but its functionality is broader. +Allows executing `JOIN` with an array or nested data structure. The intent is similar to the [arrayJoin](functions/array_join.md#functions_arrayjoin) function, but its functionality is broader. ``` sql SELECT @@ -168,14 +168,14 @@ FROM You can specify only a single `ARRAY JOIN` clause in a query. -When running the `ARRAY JOIN`, there is an optimization of the query execution order. Although the `ARRAY JOIN` must be always specified before the `WHERE/PREWHERE` clause, it can be performed as before the `WHERE/PREWHERE` (if its result is needed in this clause), as after completing it (to reduce the volume of calculations). The processing order is controlled by the query optimizer. +The query execution order is optimized when running `ARRAY JOIN`. Although `ARRAY JOIN` must always be specified before the `WHERE/PREWHERE` clause, it can be performed either before `WHERE/PREWHERE` (if the result is needed in this clause), or after completing it (to reduce the volume of calculations). The processing order is controlled by the query optimizer. Supported types of `ARRAY JOIN` are listed below: -- `ARRAY JOIN` - Executing `JOIN` with an array or nested data structure. Empty arrays are not included in the result. -- `LEFT ARRAY JOIN` - Unlike `ARRAY JOIN`, when using the `LEFT ARRAY JOIN` the result contains the rows with empty arrays. The value for an empty array is set to default value for an array element type (usually 0, empty string or NULL). +- `ARRAY JOIN` - In this case, empty arrays are not included in the result of `JOIN`. +- `LEFT ARRAY JOIN` - The result of `JOIN` contains rows with empty arrays. The value for an empty array is set to the default value for the array element type (usually 0, empty string or NULL). -Examples below demonstrate the usage of the `ARRAY JOIN` clause. Let's create a table with an [Array](../data_types/array.md) type column and insert values into it: +The examples below demonstrate the usage of the `ARRAY JOIN` and `LEFT ARRAY JOIN` clauses. Let's create a table with an [Array](../data_types/array.md) type column and insert values into it: ``` sql CREATE TABLE arrays_test @@ -195,7 +195,7 @@ VALUES ('Hello', [1,2]), ('World', [3,4,5]), ('Goodbye', []); └─────────────┴─────────┘ ``` -The first example shows using the `ARRAY JOIN` clause: +The example below uses the `ARRAY JOIN` clause: ``` sql SELECT s, arr @@ -212,7 +212,7 @@ ARRAY JOIN arr; └───────┴─────┘ ``` -The second example shows using the `LEFT ARRAY JOIN` clause: +The next example uses the `LEFT ARRAY JOIN` clause: ``` sql SELECT s, arr @@ -230,7 +230,27 @@ LEFT ARRAY JOIN arr; └─────────────┴─────┘ ``` -The next example demonstrates using the `ARRAY JOIN` with the external array: +#### Using Aliases + +An alias can be specified for an array in the `ARRAY JOIN` clause. In this case, an array item can be accessed by this alias, but the array itself is accessed by the original name. Example: + +``` sql +SELECT s, arr, a +FROM arrays_test +ARRAY JOIN arr AS a; +``` + +``` +┌─s─────┬─arr─────┬─a─┐ +│ Hello │ [1,2] │ 1 │ +│ Hello │ [1,2] │ 2 │ +│ World │ [3,4,5] │ 3 │ +│ World │ [3,4,5] │ 4 │ +│ World │ [3,4,5] │ 5 │ +└───────┴─────────┴───┘ +``` + +Using aliases, you can perform `ARRAY JOIN` with an external array. For example: ``` sql SELECT s, arr_external @@ -252,27 +272,7 @@ ARRAY JOIN [1, 2, 3] AS arr_external; └─────────────┴──────────────┘ ``` -#### Using Aliases - -An alias can be specified for an array in the `ARRAY JOIN` clause. In this case, an array item can be accessed by this alias, but the array itself by the original name. Example: - -``` sql -SELECT s, arr, a -FROM arrays_test -ARRAY JOIN arr AS a; -``` - -``` -┌─s─────┬─arr─────┬─a─┐ -│ Hello │ [1,2] │ 1 │ -│ Hello │ [1,2] │ 2 │ -│ World │ [3,4,5] │ 3 │ -│ World │ [3,4,5] │ 4 │ -│ World │ [3,4,5] │ 5 │ -└───────┴─────────┴───┘ -``` - -Multiple arrays of the same size can be comma-separated in the `ARRAY JOIN` clause. In this case, `JOIN` is performed with them simultaneously (the direct sum, not the cartesian product). Example: +Multiple arrays can be comma-separated in the `ARRAY JOIN` clause. In this case, `JOIN` is performed with them simultaneously (the direct sum, not the cartesian product). Note that all the arrays must have the same size. Example: ``` sql SELECT s, arr, a, num, mapped @@ -290,6 +290,8 @@ ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num, arrayMap(x -> x + 1, arr) AS ma └───────┴─────────┴───┴─────┴────────┘ ``` +The example below uses the [arrayEnumerate](functions/array_functions.md#array_functions-arrayenumerate) function: + ``` sql SELECT s, arr, a, num, arrayEnumerate(arr) FROM arrays_test @@ -308,7 +310,7 @@ ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num; #### ARRAY JOIN With Nested Data Structure -`ARRAY JOIN` also works with [nested data structure](../data_types/nested_data_structures/nested.md). Example: +`ARRAY `JOIN`` also works with [nested data structures](../data_types/nested_data_structures/nested.md). Example: ``` sql CREATE TABLE nested_test @@ -401,7 +403,7 @@ ARRAY JOIN nest AS n; └───────┴─────┴─────┴─────────┴────────────┘ ``` -The example of using the [arrayEnumerate](functions/array_functions.md#array_functions-arrayenumerate) function: +Example of using the [arrayEnumerate](functions/array_functions.md#array_functions-arrayenumerate) function: ``` sql SELECT s, `n.x`, `n.y`, `nest.x`, `nest.y`, num @@ -444,7 +446,7 @@ The table names can be specified instead of `` and ` +FROM +[LEFT] ARRAY JOIN +[WHERE|PREWHERE ] +... ``` -:) CREATE TABLE arrays_test (s String, arr Array(UInt8)) ENGINE = Memory +В запросе может быть указано не более одной секции `ARRAY JOIN`. + +При использовании `ARRAY JOIN`, порядок выполнения запроса оптимизируется. Несмотря на то что секция `ARRAY JOIN` всегда указывается перед выражением `WHERE / PREWHERE`, преобразование `JOIN` может быть выполнено как до выполнения выражения `WHERE / PREWHERE` (если результат необходим в этом выражении), так и после (чтобы уменьшить объем расчетов). Порядок обработки контролируется оптимизатором запросов. + +Секция `ARRAY JOIN` поддерживает следующие формы записи: + +- `ARRAY JOIN` — в этом случае результат `JOIN` не будет содержать пустые массивы; +- `LEFT ARRAY JOIN` — пустые массивы попадут в результат выполнения `JOIN`. В качестве значения для пустых массивов устанавливается значение по умолчанию. Обычно это 0, пустая строка или NULL, в зависимости от типа элементов массива. + +Рассмотрим примеры использования `ARRAY JOIN` и `LEFT ARRAY JOIN`. Для начала создадим таблицу, содержащую столбец с типом [Array](../data_types/array.md), и добавим в него значение: + +``` sql CREATE TABLE arrays_test ( s String, arr Array(UInt8) -) ENGINE = Memory +) ENGINE = Memory; -Ok. +INSERT INTO arrays_test +VALUES ('Hello', [1,2]), ('World', [3,4,5]), ('Goodbye', []); +``` +``` +┌─s───────────┬─arr─────┐ +│ Hello │ [1,2] │ +│ World │ [3,4,5] │ +│ Goodbye │ [] │ +└─────────────┴─────────┘ +``` -0 rows in set. Elapsed: 0.001 sec. - -:) INSERT INTO arrays_test VALUES ('Hello', [1,2]), ('World', [3,4,5]), ('Goodbye', []) - -INSERT INTO arrays_test VALUES - -Ok. - -3 rows in set. Elapsed: 0.001 sec. - -:) SELECT * FROM arrays_test - -SELECT * -FROM arrays_test - -┌─s───────┬─arr─────┐ -│ Hello │ [1,2] │ -│ World │ [3,4,5] │ -│ Goodbye │ [] │ -└─────────┴─────────┘ - -3 rows in set. Elapsed: 0.001 sec. - -:) SELECT s, arr FROM arrays_test ARRAY JOIN arr +В примере ниже используется `ARRAY JOIN`: +``` sql SELECT s, arr FROM arrays_test -ARRAY JOIN arr - +ARRAY JOIN arr; +``` +``` ┌─s─────┬─arr─┐ │ Hello │ 1 │ │ Hello │ 2 │ @@ -212,19 +215,37 @@ ARRAY JOIN arr │ World │ 4 │ │ World │ 5 │ └───────┴─────┘ - -5 rows in set. Elapsed: 0.001 sec. ``` -Для массива в секции ARRAY JOIN может быть указан алиас. В этом случае, элемент массива будет доступен под этим алиасом, а сам массив - под исходным именем. Пример: +Следующий пример использует `LEFT ARRAY JOIN`: +``` sql +SELECT s, arr +FROM arrays_test +LEFT ARRAY JOIN arr; ``` -:) SELECT s, arr, a FROM arrays_test ARRAY JOIN arr AS a +``` +┌─s───────────┬─arr─┐ +│ Hello │ 1 │ +│ Hello │ 2 │ +│ World │ 3 │ +│ World │ 4 │ +│ World │ 5 │ +│ Goodbye │ 0 │ +└─────────────┴─────┘ +``` +#### Использование алиасов + +Для массива в секции `ARRAY JOIN` может быть указан алиас. В этом случае, элемент массива будет доступен под этим алиасом, а сам массив — под исходным именем. Пример: + +``` sql SELECT s, arr, a FROM arrays_test -ARRAY JOIN arr AS a +ARRAY JOIN arr AS a; +``` +``` ┌─s─────┬─arr─────┬─a─┐ │ Hello │ [1,2] │ 1 │ │ Hello │ [1,2] │ 2 │ @@ -232,19 +253,39 @@ ARRAY JOIN arr AS a │ World │ [3,4,5] │ 4 │ │ World │ [3,4,5] │ 5 │ └───────┴─────────┴───┘ - -5 rows in set. Elapsed: 0.001 sec. ``` -В секции ARRAY JOIN может быть указано несколько массивов одинаковых размеров через запятую. В этом случае, JOIN делается с ними одновременно (прямая сумма, а не прямое произведение). Пример: +Используя алиасы, можно выполнять `JOIN` с внешними массивами: + +``` sql +SELECT s, arr_external +FROM arrays_test +ARRAY JOIN [1, 2, 3] AS arr_external; +``` ``` -:) SELECT s, arr, a, num, mapped FROM arrays_test ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num, arrayMap(x -> x + 1, arr) AS mapped +┌─s───────────┬─arr_external─┐ +│ Hello │ 1 │ +│ Hello │ 2 │ +│ Hello │ 3 │ +│ World │ 1 │ +│ World │ 2 │ +│ World │ 3 │ +│ Goodbye │ 1 │ +│ Goodbye │ 2 │ +│ Goodbye │ 3 │ +└─────────────┴──────────────┘ +``` +В секции `ARRAY JOIN` можно указать через запятую сразу несколько массивов. В этом случае, `JOIN` делается с ними одновременно (прямая сумма, а не прямое произведение). Обратите внимание, массивы должны быть одинаковых размеров. Примеры: + +``` sql SELECT s, arr, a, num, mapped FROM arrays_test -ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num, arrayMap(lambda(tuple(x), plus(x, 1)), arr) AS mapped +ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num, arrayMap(x -> x + 1, arr) AS mapped; +``` +``` ┌─s─────┬─arr─────┬─a─┬─num─┬─mapped─┐ │ Hello │ [1,2] │ 1 │ 1 │ 2 │ │ Hello │ [1,2] │ 2 │ 2 │ 3 │ @@ -252,15 +293,17 @@ ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num, arrayMap(lambda(tuple(x), plus( │ World │ [3,4,5] │ 4 │ 2 │ 5 │ │ World │ [3,4,5] │ 5 │ 3 │ 6 │ └───────┴─────────┴───┴─────┴────────┘ +``` -5 rows in set. Elapsed: 0.002 sec. - -:) SELECT s, arr, a, num, arrayEnumerate(arr) FROM arrays_test ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num +В примере ниже используется функция [arrayEnumerate](functions/array_functions.md#array_functions-arrayenumerate): +``` sql SELECT s, arr, a, num, arrayEnumerate(arr) FROM arrays_test -ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num +ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num; +``` +``` ┌─s─────┬─arr─────┬─a─┬─num─┬─arrayEnumerate(arr)─┐ │ Hello │ [1,2] │ 1 │ 1 │ [1,2] │ │ Hello │ [1,2] │ 2 │ 2 │ [1,2] │ @@ -268,54 +311,40 @@ ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num │ World │ [3,4,5] │ 4 │ 2 │ [1,2,3] │ │ World │ [3,4,5] │ 5 │ 3 │ [1,2,3] │ └───────┴─────────┴───┴─────┴─────────────────────┘ - -5 rows in set. Elapsed: 0.002 sec. ``` -ARRAY JOIN также работает с вложенными структурами данных. Пример: +#### ARRAY JOIN с вложенными структурами данных -``` -:) CREATE TABLE nested_test (s String, nest Nested(x UInt8, y UInt32)) ENGINE = Memory +`ARRAY JOIN` также работает с [вложенными структурами данных](../data_types/nested_data_structures/nested.md). Пример: +``` sql CREATE TABLE nested_test ( s String, nest Nested( x UInt8, y UInt32) -) ENGINE = Memory +) ENGINE = Memory; -Ok. - -0 rows in set. Elapsed: 0.006 sec. - -:) INSERT INTO nested_test VALUES ('Hello', [1,2], [10,20]), ('World', [3,4,5], [30,40,50]), ('Goodbye', [], []) - -INSERT INTO nested_test VALUES - -Ok. - -3 rows in set. Elapsed: 0.001 sec. - -:) SELECT * FROM nested_test - -SELECT * -FROM nested_test +INSERT INTO nested_test +VALUES ('Hello', [1,2], [10,20]), ('World', [3,4,5], [30,40,50]), ('Goodbye', [], []); +``` +``` ┌─s───────┬─nest.x──┬─nest.y─────┐ │ Hello │ [1,2] │ [10,20] │ │ World │ [3,4,5] │ [30,40,50] │ │ Goodbye │ [] │ [] │ └─────────┴─────────┴────────────┘ +``` -3 rows in set. Elapsed: 0.001 sec. - -:) SELECT s, nest.x, nest.y FROM nested_test ARRAY JOIN nest - +``` sql SELECT s, `nest.x`, `nest.y` FROM nested_test -ARRAY JOIN nest +ARRAY JOIN nest; +``` +``` ┌─s─────┬─nest.x─┬─nest.y─┐ │ Hello │ 1 │ 10 │ │ Hello │ 2 │ 20 │ @@ -323,19 +352,17 @@ ARRAY JOIN nest │ World │ 4 │ 40 │ │ World │ 5 │ 50 │ └───────┴────────┴────────┘ - -5 rows in set. Elapsed: 0.001 sec. ``` -При указании имени вложенной структуры данных в ARRAY JOIN, смысл такой же, как ARRAY JOIN со всеми элементами-массивами, из которых она состоит. Пример: - -``` -:) SELECT s, nest.x, nest.y FROM nested_test ARRAY JOIN nest.x, nest.y +При указании имени вложенной структуры данных в `ARRAY JOIN`, смысл такой же, как `ARRAY JOIN` со всеми элементами-массивами, из которых она состоит. Пример: +``` sql SELECT s, `nest.x`, `nest.y` FROM nested_test -ARRAY JOIN `nest.x`, `nest.y` +ARRAY JOIN `nest.x`, `nest.y`; +``` +``` ┌─s─────┬─nest.x─┬─nest.y─┐ │ Hello │ 1 │ 10 │ │ Hello │ 2 │ 20 │ @@ -343,19 +370,17 @@ ARRAY JOIN `nest.x`, `nest.y` │ World │ 4 │ 40 │ │ World │ 5 │ 50 │ └───────┴────────┴────────┘ - -5 rows in set. Elapsed: 0.001 sec. ``` Такой вариант тоже имеет смысл: -``` -:) SELECT s, nest.x, nest.y FROM nested_test ARRAY JOIN nest.x - +``` sql SELECT s, `nest.x`, `nest.y` FROM nested_test -ARRAY JOIN `nest.x` +ARRAY JOIN `nest.x`; +``` +``` ┌─s─────┬─nest.x─┬─nest.y─────┐ │ Hello │ 1 │ [10,20] │ │ Hello │ 2 │ [10,20] │ @@ -363,19 +388,17 @@ ARRAY JOIN `nest.x` │ World │ 4 │ [30,40,50] │ │ World │ 5 │ [30,40,50] │ └───────┴────────┴────────────┘ - -5 rows in set. Elapsed: 0.001 sec. ``` -Алиас для вложенной структуры данных можно использовать, чтобы выбрать как результат JOIN-а, так и исходный массив. Пример: - -``` -:) SELECT s, n.x, n.y, nest.x, nest.y FROM nested_test ARRAY JOIN nest AS n +Алиас для вложенной структуры данных можно использовать, чтобы выбрать как результат `JOIN`-а, так и исходный массив. Пример: +``` sql SELECT s, `n.x`, `n.y`, `nest.x`, `nest.y` FROM nested_test -ARRAY JOIN nest AS n +ARRAY JOIN nest AS n; +``` +``` ┌─s─────┬─n.x─┬─n.y─┬─nest.x──┬─nest.y─────┐ │ Hello │ 1 │ 10 │ [1,2] │ [10,20] │ │ Hello │ 2 │ 20 │ [1,2] │ [10,20] │ @@ -383,19 +406,17 @@ ARRAY JOIN nest AS n │ World │ 4 │ 40 │ [3,4,5] │ [30,40,50] │ │ World │ 5 │ 50 │ [3,4,5] │ [30,40,50] │ └───────┴─────┴─────┴─────────┴────────────┘ - -5 rows in set. Elapsed: 0.001 sec. ``` -Пример использования функции arrayEnumerate: - -``` -:) SELECT s, n.x, n.y, nest.x, nest.y, num FROM nested_test ARRAY JOIN nest AS n, arrayEnumerate(nest.x) AS num +Пример использования функции [arrayEnumerate](functions/array_functions.md#array_functions-arrayenumerate): +``` sql SELECT s, `n.x`, `n.y`, `nest.x`, `nest.y`, num FROM nested_test -ARRAY JOIN nest AS n, arrayEnumerate(`nest.x`) AS num +ARRAY JOIN nest AS n, arrayEnumerate(`nest.x`) AS num; +``` +``` ┌─s─────┬─n.x─┬─n.y─┬─nest.x──┬─nest.y─────┬─num─┐ │ Hello │ 1 │ 10 │ [1,2] │ [10,20] │ 1 │ │ Hello │ 2 │ 20 │ [1,2] │ [10,20] │ 2 │ @@ -403,16 +424,8 @@ ARRAY JOIN nest AS n, arrayEnumerate(`nest.x`) AS num │ World │ 4 │ 40 │ [3,4,5] │ [30,40,50] │ 2 │ │ World │ 5 │ 50 │ [3,4,5] │ [30,40,50] │ 3 │ └───────┴─────┴─────┴─────────┴────────────┴─────┘ - -5 rows in set. Elapsed: 0.002 sec. ``` -В запросе может быть указано не более одной секции ARRAY JOIN. - -Соответствующее преобразование может выполняться как до секции WHERE/PREWHERE (если его результат нужен в этой секции), так и после выполнения WHERE/PREWHERE (чтобы уменьшить объём вычислений). - - - ### Секция JOIN {#select-join} Соединяет данные в привычном для [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) смысле. From 3bbe57adac23d9a599e56f7a9c0bf3665b5a366e Mon Sep 17 00:00:00 2001 From: hcz Date: Mon, 22 Apr 2019 17:31:01 +0800 Subject: [PATCH 18/37] Rename AggregateFunctionsStatisticsSimple.cpp to AggregateFunctionStatisticsSimple.cpp --- ...StatisticsSimple.cpp => AggregateFunctionStatisticsSimple.cpp} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename dbms/src/AggregateFunctions/{AggregateFunctionsStatisticsSimple.cpp => AggregateFunctionStatisticsSimple.cpp} (100%) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionsStatisticsSimple.cpp b/dbms/src/AggregateFunctions/AggregateFunctionStatisticsSimple.cpp similarity index 100% rename from dbms/src/AggregateFunctions/AggregateFunctionsStatisticsSimple.cpp rename to dbms/src/AggregateFunctions/AggregateFunctionStatisticsSimple.cpp From 7a6365dbf1436e82ddd2bd1c3642040a95e88afe Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Apr 2019 13:56:10 +0300 Subject: [PATCH 19/37] Increase sleep time in flappy test and remove jokes --- ...ndex_granularity_replicated_merge_tree.sql | 6 +++--- .../queries/0_stateless/00933_ttl_simple.sql | 19 ------------------- 2 files changed, 3 insertions(+), 22 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql b/dbms/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql index 0ac96ce751b..df9d791d824 100644 --- a/dbms/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql +++ b/dbms/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql @@ -35,7 +35,7 @@ SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database='test' and active=1; -SELECT sleep(0.5) Format Null; +SELECT sleep(0.7) Format Null; OPTIMIZE TABLE test.zero_rows_per_granule2 FINAL; @@ -92,7 +92,7 @@ SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database='test' and active=1; -SELECT sleep(0.5) Format Null; +SELECT sleep(0.7) Format Null; OPTIMIZE TABLE test.four_rows_per_granule2 FINAL; @@ -180,7 +180,7 @@ ATTACH TABLE test.adaptive_granularity_alter1; INSERT INTO test.adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 100, 1000, 'aaaa'), ('2018-05-16', 101, 3000, 'bbbb'), ('2018-05-17', 102, 5000, 'cccc'), ('2018-05-19', 103, 7000, 'dddd'); -SELECT sleep(0.5) Format Null; +SELECT sleep(0.7) Format Null; OPTIMIZE TABLE test.adaptive_granularity_alter1 FINAL; diff --git a/dbms/tests/queries/0_stateless/00933_ttl_simple.sql b/dbms/tests/queries/0_stateless/00933_ttl_simple.sql index 9e62e46e0df..62b320cc0b0 100644 --- a/dbms/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/dbms/tests/queries/0_stateless/00933_ttl_simple.sql @@ -47,22 +47,3 @@ create table ttl_00933_1 (d DateTime, a Int ttl toDateTime(1)) engine = MergeTre create table ttl_00933_1 (d DateTime, a Int ttl d - d) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 450 } drop table if exists ttl_00933_1; - -/* - -Alexey Milovidov, [17.04.19 20:09] -sleep(0.7) -sleep(1.1) -- почему? @Alesapin - -Alexander Sapin, [17.04.19 23:16] -[In reply to Alexey Milovidov] -1.1 по логике теста, я попробовал с 0.5 и у меня флапнуло. С 1 не флапало, но работало долго. Попробовал 0.7 и тоже не флапает. - -Alexey Milovidov, [17.04.19 23:18] -Слабо такой комментарий добавить прямо в тест? :) - -Alexander Sapin, [17.04.19 23:20] -как-то неловко :) - -*/ From a9bc78d55933ec51dcd1d54585ad7872db345548 Mon Sep 17 00:00:00 2001 From: proller Date: Mon, 22 Apr 2019 14:48:37 +0300 Subject: [PATCH 20/37] Dont use test. db --- dbms/tests/queries/0_stateless/00722_inner_join.sql | 7 +++---- .../queries/0_stateless/00860_unknown_identifier_bug.sql | 6 ++---- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00722_inner_join.sql b/dbms/tests/queries/0_stateless/00722_inner_join.sql index 5cc0983755b..eee164ed1e1 100644 --- a/dbms/tests/queries/0_stateless/00722_inner_join.sql +++ b/dbms/tests/queries/0_stateless/00722_inner_join.sql @@ -1,6 +1,5 @@ -CREATE DATABASE IF NOT EXISTS test; -DROP TABLE IF EXISTS test.one; -CREATE TABLE test.one(dummy UInt8) ENGINE = Memory; +DROP TABLE IF EXISTS one; +CREATE TABLE one(dummy UInt8) ENGINE = Memory; SELECT database, t.name FROM system.tables AS t @@ -91,4 +90,4 @@ SELECT count() JOIN system.databases AS db ON db.name = t.database WHERE t.name = 'one'; -DROP TABLE test.one; +DROP TABLE one; diff --git a/dbms/tests/queries/0_stateless/00860_unknown_identifier_bug.sql b/dbms/tests/queries/0_stateless/00860_unknown_identifier_bug.sql index cd5f45fd3fc..bbcd3de8f20 100644 --- a/dbms/tests/queries/0_stateless/00860_unknown_identifier_bug.sql +++ b/dbms/tests/queries/0_stateless/00860_unknown_identifier_bug.sql @@ -1,5 +1,3 @@ -USE test; - DROP TABLE IF EXISTS appointment_events; CREATE TABLE appointment_events ( @@ -29,11 +27,11 @@ SELECT A._appointment_id, A._vacancy_id, A._set_at, A._job_requisition_id -FROM test.appointment_events A ANY +FROM appointment_events A ANY LEFT JOIN (SELECT _appointment_id, MAX(_set_at) AS max_set_at - FROM test.appointment_events + FROM appointment_events WHERE _status in ('Created', 'Transferred') GROUP BY _appointment_id ) B USING _appointment_id WHERE A._set_at = B.max_set_at; From d479836362579fea2dffa6119d621fb8447f510d Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 18 Apr 2019 18:52:18 +0300 Subject: [PATCH 21/37] Add test on lost messages --- .../helpers/docker_compose_kafka.yml | 2 +- .../integration/test_storage_kafka/test.py | 49 ++++++++++++++++++- 2 files changed, 49 insertions(+), 2 deletions(-) diff --git a/dbms/tests/integration/helpers/docker_compose_kafka.yml b/dbms/tests/integration/helpers/docker_compose_kafka.yml index bed537a9760..8fea4faa272 100644 --- a/dbms/tests/integration/helpers/docker_compose_kafka.yml +++ b/dbms/tests/integration/helpers/docker_compose_kafka.yml @@ -12,7 +12,7 @@ services: - label:disable kafka1: - image: confluentinc/cp-kafka:4.1.0 + image: confluentinc/cp-kafka:5.2.0 hostname: kafka1 ports: - "9092:9092" diff --git a/dbms/tests/integration/test_storage_kafka/test.py b/dbms/tests/integration/test_storage_kafka/test.py index 0258d38b8ab..c67b95c1e83 100644 --- a/dbms/tests/integration/test_storage_kafka/test.py +++ b/dbms/tests/integration/test_storage_kafka/test.py @@ -7,7 +7,8 @@ from helpers.test_tools import TSV import json import subprocess -from kafka import KafkaProducer +import kafka.errors +from kafka import KafkaAdminClient, KafkaProducer from google.protobuf.internal.encoder import _VarintBytes """ @@ -318,6 +319,52 @@ def test_kafka_materialized_view(kafka_cluster): ''') +def test_kafka_flush_on_big_message(kafka_cluster): + # Create batchs of messages of size ~100Kb + kafka_messages = 10000 + batch_messages = 1000 + messages = [json.dumps({'key': i, 'value': 'x' * 100}) * batch_messages for i in range(kafka_messages)] + kafka_produce('flush', messages) + + instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.kafka (key UInt64, value String) + ENGINE = Kafka + SETTINGS + kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'flush', + kafka_group_name = 'flush', + kafka_format = 'JSONEachRow', + kafka_max_block_size = 10; + CREATE TABLE test.view (key UInt64, value String) + ENGINE = MergeTree + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka; + ''') + + client = KafkaAdminClient(bootstrap_servers="localhost:9092") + received = False + while not received: + try: + offsets = client.list_consumer_group_offsets('flush') + for topic, offset in offsets.items(): + if topic.topic == 'flush' and offset.offset == kafka_messages: + received = True + break + except kafka.errors.GroupCoordinatorNotAvailableError: + continue + + for _ in range(20): + time.sleep(1) + result = instance.query('SELECT count() FROM test.view') + if int(result) == kafka_messages*batch_messages: + break + + assert int(result) == kafka_messages*batch_messages, 'ClickHouse lost some messages: {}'.format(result) + + if __name__ == '__main__': cluster.start() raw_input("Cluster created, press any key to destroy...") From 81259c0b522267805c803ccbc53ec47ddc36efd6 Mon Sep 17 00:00:00 2001 From: Alexander Mochalin Date: Mon, 22 Apr 2019 17:43:31 +0500 Subject: [PATCH 22/37] fix(docs): typo (#5078) --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index e999b300de9..c53c71bb8f4 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -236,7 +236,7 @@ Ok. ## preferred_block_size_bytes -Служит для тех же целей что и `max_block_size`, но задает реккомедуемый размер блоков в байтах, выбирая адаптивное количество строк в блоке. +Служит для тех же целей что и `max_block_size`, но задает рекомедуемый размер блоков в байтах, выбирая адаптивное количество строк в блоке. При этом размер блока не может быть более `max_block_size` строк. По умолчанию: 1,000,000. Работает только при чтении из MergeTree-движков. From 94bba1918f63c325202420e313024debbce5c67c Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Mon, 22 Apr 2019 16:23:05 +0300 Subject: [PATCH 23/37] Do not drop Kafka consumer buffers after closing stream --- .../Storages/Kafka/KafkaBlockInputStream.cpp | 31 +++--------- .../Storages/Kafka/KafkaBlockInputStream.h | 4 +- .../Kafka/ReadBufferFromKafkaConsumer.cpp | 27 +++++++++- .../Kafka/ReadBufferFromKafkaConsumer.h | 13 +++-- dbms/src/Storages/Kafka/StorageKafka.cpp | 50 +++++++++++-------- dbms/src/Storages/Kafka/StorageKafka.h | 18 +++---- 6 files changed, 82 insertions(+), 61 deletions(-) diff --git a/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp b/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp index c511a1053b3..2cabb9c431f 100644 --- a/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp +++ b/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp @@ -24,36 +24,21 @@ KafkaBlockInputStream::~KafkaBlockInputStream() return; if (broken) - { - LOG_TRACE(storage.log, "Re-joining claimed consumer after failure"); - consumer->unsubscribe(); - } + buffer->subBufferAs()->unsubscribe(); - storage.pushConsumer(consumer); + storage.pushBuffer(buffer); } void KafkaBlockInputStream::readPrefixImpl() { - consumer = storage.tryClaimConsumer(context.getSettingsRef().queue_max_wait_ms.totalMilliseconds()); - claimed = !!consumer; + buffer = storage.tryClaimBuffer(context.getSettingsRef().queue_max_wait_ms.totalMilliseconds()); + claimed = !!buffer; - if (!consumer) - consumer = std::make_shared(storage.createConsumerConfiguration()); + if (!buffer) + buffer = storage.createBuffer(); - // While we wait for an assignment after subscribtion, we'll poll zero messages anyway. - // If we're doing a manual select then it's better to get something after a wait, then immediate nothing. - if (consumer->get_subscription().empty()) - { - using namespace std::chrono_literals; + buffer->subBufferAs()->subscribe(storage.topics); - consumer->pause(); // don't accidentally read any messages - consumer->subscribe(storage.topics); - consumer->poll(5s); - consumer->resume(); - } - - buffer = std::make_unique( - new ReadBufferFromKafkaConsumer(consumer, storage.log, max_block_size), storage.row_delimiter); addChild(FormatFactory::instance().getInput(storage.format_name, *buffer, storage.getSampleBlock(), context, max_block_size)); broken = true; @@ -66,4 +51,4 @@ void KafkaBlockInputStream::readSuffixImpl() broken = false; } -} // namespace DB +} diff --git a/dbms/src/Storages/Kafka/KafkaBlockInputStream.h b/dbms/src/Storages/Kafka/KafkaBlockInputStream.h index 29d013c183d..1b6c8b8ae25 100644 --- a/dbms/src/Storages/Kafka/KafkaBlockInputStream.h +++ b/dbms/src/Storages/Kafka/KafkaBlockInputStream.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include @@ -27,8 +26,7 @@ private: Context context; UInt64 max_block_size; - ConsumerPtr consumer; - std::unique_ptr buffer; + BufferPtr buffer; bool broken = true, claimed = false; }; diff --git a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp index 948662a9f93..fa5eb453e92 100644 --- a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp +++ b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp @@ -2,10 +2,11 @@ namespace DB { + namespace { const auto READ_POLL_MS = 500; /// How long to wait for a batch of messages. -} // namespace +} void ReadBufferFromKafkaConsumer::commit() { @@ -13,10 +14,32 @@ void ReadBufferFromKafkaConsumer::commit() return; auto & previous = *std::prev(current); + LOG_TRACE(log, "Committing message with offset " << previous.get_offset()); consumer->async_commit(previous); } +void ReadBufferFromKafkaConsumer::subscribe(const Names & topics) +{ + // While we wait for an assignment after subscribtion, we'll poll zero messages anyway. + // If we're doing a manual select then it's better to get something after a wait, then immediate nothing. + if (consumer->get_subscription().empty()) + { + using namespace std::chrono_literals; + + consumer->pause(); // don't accidentally read any messages + consumer->subscribe(topics); + consumer->poll(5s); + consumer->resume(); + } +} + +void ReadBufferFromKafkaConsumer::unsubscribe() +{ + LOG_TRACE(log, "Re-joining claimed consumer after failure"); + consumer->unsubscribe(); +} + /// Do commit messages implicitly after we processed the previous batch. bool ReadBufferFromKafkaConsumer::nextImpl() { @@ -50,4 +73,4 @@ bool ReadBufferFromKafkaConsumer::nextImpl() return true; } -} // namespace DB +} diff --git a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h index 8a3ebcbc2ef..f7b49d955de 100644 --- a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h +++ b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h @@ -1,14 +1,18 @@ #pragma once -#include +#include +#include #include #include namespace DB { + +using BufferPtr = std::shared_ptr; using ConsumerPtr = std::shared_ptr; + class ReadBufferFromKafkaConsumer : public ReadBuffer { public: @@ -17,8 +21,9 @@ public: { } - // Commit all processed messages. - void commit(); + void commit(); // Commit all processed messages. + void subscribe(const Names & topics); // Subscribe internal consumer to topics. + void unsubscribe(); // Unsubscribe internal consumer in case of failure. private: using Messages = std::vector; @@ -33,4 +38,4 @@ private: bool nextImpl() override; }; -} // namespace DB +} diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 5785e65d4d9..1e1ee6bb695 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -81,7 +81,7 @@ StorageKafka::StorageKafka( row_delimiter(row_delimiter_), schema_name(global_context.getMacros()->expand(schema_name_)), num_consumers(num_consumers_), max_block_size(max_block_size_), log(&Logger::get("StorageKafka (" + table_name_ + ")")), - semaphore(0, num_consumers_), mutex(), consumers(), + semaphore(0, num_consumers_), skip_broken(skip_broken_) { task = global_context.getSchedulePool().createTask(log->name(), [this]{ streamThread(); }); @@ -124,12 +124,8 @@ void StorageKafka::startup() { for (size_t i = 0; i < num_consumers; ++i) { - // Create a consumer and subscribe to topics - auto consumer = std::make_shared(createConsumerConfiguration()); - consumer->subscribe(topics); - - // Make consumer available - pushConsumer(consumer); + // Make buffer available + pushBuffer(createBuffer()); ++num_created_consumers; } @@ -146,8 +142,8 @@ void StorageKafka::shutdown() // Close all consumers for (size_t i = 0; i < num_created_consumers; ++i) { - auto consumer = claimConsumer(); - // FIXME: not sure if really close consumers here, and if we really need to close them here. + auto buffer = claimBuffer(); + // FIXME: not sure if we really close consumers here, and if we really need to close them here. } LOG_TRACE(log, "Waiting for cleanup"); @@ -203,14 +199,29 @@ cppkafka::Configuration StorageKafka::createConsumerConfiguration() return conf; } -ConsumerPtr StorageKafka::claimConsumer() +BufferPtr StorageKafka::createBuffer() { - return tryClaimConsumer(-1L); + // Create a consumer and subscribe to topics + auto consumer = std::make_shared(createConsumerConfiguration()); + consumer->subscribe(topics); + + // Limit the number of batched messages to allow early cancellations + const Settings & settings = global_context.getSettingsRef(); + size_t batch_size = max_block_size; + if (!batch_size) + batch_size = settings.max_block_size.value; + + return std::make_shared(new ReadBufferFromKafkaConsumer(consumer, log, batch_size), row_delimiter); } -ConsumerPtr StorageKafka::tryClaimConsumer(long wait_ms) +BufferPtr StorageKafka::claimBuffer() { - // Wait for the first free consumer + return tryClaimBuffer(-1L); +} + +BufferPtr StorageKafka::tryClaimBuffer(long wait_ms) +{ + // Wait for the first free buffer if (wait_ms >= 0) { if (!semaphore.tryWait(wait_ms)) @@ -219,17 +230,17 @@ ConsumerPtr StorageKafka::tryClaimConsumer(long wait_ms) else semaphore.wait(); - // Take the first available consumer from the list + // Take the first available buffer from the list std::lock_guard lock(mutex); - auto consumer = consumers.back(); - consumers.pop_back(); - return consumer; + auto buffer = buffers.back(); + buffers.pop_back(); + return buffer; } -void StorageKafka::pushConsumer(ConsumerPtr consumer) +void StorageKafka::pushBuffer(BufferPtr buffer) { std::lock_guard lock(mutex); - consumers.push_back(consumer); + buffers.push_back(buffer); semaphore.set(); } @@ -303,7 +314,6 @@ bool StorageKafka::streamToViews() insert->table = table_name; insert->no_destination = true; // Only insert into dependent views - // Limit the number of batched messages to allow early cancellations const Settings & settings = global_context.getSettingsRef(); size_t block_size = max_block_size; if (block_size == 0) diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index b8d09bb2da1..c2de0e01178 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -14,15 +15,13 @@ namespace DB { -using ConsumerPtr = std::shared_ptr; - /** Implements a Kafka queue table engine that can be used as a persistent queue / buffer, * or as a basic building block for creating pipelines with a continuous insertion / ETL. */ class StorageKafka : public ext::shared_ptr_helper, public IStorage { -friend class KafkaBlockInputStream; -friend class KafkaBlockOutputStream; + friend class KafkaBlockInputStream; + friend class KafkaBlockOutputStream; public: std::string getName() const override { return "Kafka"; } @@ -40,7 +39,7 @@ public: size_t max_block_size, unsigned num_streams) override; - void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) override + void rename(const String & /* new_path_to_db */, const String & new_database_name, const String & new_table_name) override { table_name = new_table_name; database_name = new_database_name; @@ -74,7 +73,7 @@ private: // Consumer list Poco::Semaphore semaphore; std::mutex mutex; - std::vector consumers; /// Available consumers + std::vector buffers; /// available buffers for Kafka consumers size_t skip_broken; @@ -83,9 +82,10 @@ private: std::atomic stream_cancelled{false}; cppkafka::Configuration createConsumerConfiguration(); - ConsumerPtr claimConsumer(); - ConsumerPtr tryClaimConsumer(long wait_ms); - void pushConsumer(ConsumerPtr c); + BufferPtr createBuffer(); + BufferPtr claimBuffer(); + BufferPtr tryClaimBuffer(long wait_ms); + void pushBuffer(BufferPtr buf); void streamThread(); bool streamToViews(); From 58d7d1e45195cf01fb26d8771f1f56c2c872e92f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Apr 2019 19:07:09 +0300 Subject: [PATCH 24/37] Resolved some of PVS-Studio warnings --- dbms/programs/copier/ClusterCopier.cpp | 2 +- .../performance-test/ConfigPreprocessor.cpp | 2 +- dbms/src/Common/HashTable/HashTable.h | 4 +- .../src/Common/ZooKeeper/ZooKeeperNodeCache.h | 2 +- dbms/src/Core/SortCursor.h | 2 +- .../CollapsingFinalBlockInputStream.h | 2 +- .../MergingSortedBlockInputStream.h | 2 +- .../src/DataStreams/ParallelInputsProcessor.h | 2 +- .../src/Dictionaries/ExternalQueryBuilder.cpp | 2 +- dbms/src/IO/HexWriteBuffer.h | 2 +- dbms/src/IO/readFloatText.h | 2 +- dbms/src/Interpreters/Aggregator.h | 4 +- dbms/src/Interpreters/Cluster.h | 4 +- dbms/src/Interpreters/ExpressionJIT.cpp | 2 +- dbms/src/Storages/MergeTree/BoolMask.h | 4 +- libs/libmysqlxx/include/mysqlxx/Row.h | 6 +- libs/libmysqlxx/src/Query.cpp | 3 + utils/test-data-generator/CMakeLists.txt | 6 - utils/test-data-generator/main.cpp | 378 ------------------ utils/test-data-generator/markov-model.cpp | 114 ------ 20 files changed, 25 insertions(+), 520 deletions(-) delete mode 100644 utils/test-data-generator/main.cpp delete mode 100644 utils/test-data-generator/markov-model.cpp diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 67351d9696d..75096df74ed 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -2054,7 +2054,7 @@ private: ConfigurationPtr task_cluster_initial_config; ConfigurationPtr task_cluster_current_config; - Coordination::Stat task_descprtion_current_stat; + Coordination::Stat task_descprtion_current_stat{}; std::unique_ptr task_cluster; diff --git a/dbms/programs/performance-test/ConfigPreprocessor.cpp b/dbms/programs/performance-test/ConfigPreprocessor.cpp index c448d84bc88..3ea095a5175 100644 --- a/dbms/programs/performance-test/ConfigPreprocessor.cpp +++ b/dbms/programs/performance-test/ConfigPreprocessor.cpp @@ -16,7 +16,7 @@ std::vector ConfigPreprocessor::processConfig( std::vector result; for (const auto & path : paths) { - result.emplace_back(new XMLConfiguration(path)); + result.emplace_back(XMLConfigurationPtr(new XMLConfiguration(path))); result.back()->setString("path", Poco::Path(path).absolute().toString()); } diff --git a/dbms/src/Common/HashTable/HashTable.h b/dbms/src/Common/HashTable/HashTable.h index 795c05a2a3e..f15fe22f8fd 100644 --- a/dbms/src/Common/HashTable/HashTable.h +++ b/dbms/src/Common/HashTable/HashTable.h @@ -526,7 +526,7 @@ public: { public: Reader(DB::ReadBuffer & in_) - : in(in_) + : in(in_) { } @@ -566,7 +566,7 @@ public: DB::ReadBuffer & in; Cell cell; size_t read_count = 0; - size_t size; + size_t size = 0; bool is_eof = false; bool is_initialized = false; }; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h index 73e7753cdcf..8bdb55e6ebd 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h @@ -41,7 +41,7 @@ public: { bool exists = false; std::string contents; - Coordination::Stat stat; + Coordination::Stat stat{}; }; ZNode get(const std::string & path, EventPtr watch_event); diff --git a/dbms/src/Core/SortCursor.h b/dbms/src/Core/SortCursor.h index cd427fe8ab1..5a49209cb71 100644 --- a/dbms/src/Core/SortCursor.h +++ b/dbms/src/Core/SortCursor.h @@ -29,7 +29,7 @@ struct SortCursorImpl * Cursor number (always?) equals to number of merging part. * Therefore this field can be used to determine part number of current row (see ColumnGathererStream). */ - size_t order; + size_t order = 0; using NeedCollationFlags = std::vector; diff --git a/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h b/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h index 769c8de7e2b..af049cce7c0 100644 --- a/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h +++ b/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h @@ -163,7 +163,7 @@ private: struct Cursor { MergingBlockPtr block; - size_t pos; + size_t pos = 0; Cursor() {} explicit Cursor(const MergingBlockPtr & block_, size_t pos_ = 0) : block(block_), pos(pos_) {} diff --git a/dbms/src/DataStreams/MergingSortedBlockInputStream.h b/dbms/src/DataStreams/MergingSortedBlockInputStream.h index be05783c3ed..c519914f174 100644 --- a/dbms/src/DataStreams/MergingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/MergingSortedBlockInputStream.h @@ -81,7 +81,7 @@ protected: struct RowRef { ColumnRawPtrs * columns = nullptr; - size_t row_num; + size_t row_num = 0; SharedBlockPtr shared_block; void swap(RowRef & other) diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index 9c7a1fc6928..9088323af45 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -163,7 +163,7 @@ private: struct InputData { BlockInputStreamPtr in; - size_t i; /// The source number (for debugging). + size_t i = 0; /// The source number (for debugging). InputData() {} InputData(const BlockInputStreamPtr & in_, size_t i_) : in(in_), i(i_) {} diff --git a/dbms/src/Dictionaries/ExternalQueryBuilder.cpp b/dbms/src/Dictionaries/ExternalQueryBuilder.cpp index 029229cf9a4..8f3135f23b7 100644 --- a/dbms/src/Dictionaries/ExternalQueryBuilder.cpp +++ b/dbms/src/Dictionaries/ExternalQueryBuilder.cpp @@ -298,7 +298,7 @@ ExternalQueryBuilder::composeLoadKeysQuery(const Columns & key_columns, const st composeKeyCondition(key_columns, row, out); } } - else if (method == IN_WITH_TUPLES) + else /* if (method == IN_WITH_TUPLES) */ { writeString(composeKeyTupleDefinition(), out); writeString(" IN (", out); diff --git a/dbms/src/IO/HexWriteBuffer.h b/dbms/src/IO/HexWriteBuffer.h index 80be88692e3..a68dd29065b 100644 --- a/dbms/src/IO/HexWriteBuffer.h +++ b/dbms/src/IO/HexWriteBuffer.h @@ -15,7 +15,7 @@ namespace DB class HexWriteBuffer final : public WriteBuffer { protected: - char buf[DBMS_HEX_WRITE_BUFFER_SIZE]; + char buf[DBMS_HEX_WRITE_BUFFER_SIZE]; //-V730 WriteBuffer & out; void nextImpl() override; diff --git a/dbms/src/IO/readFloatText.h b/dbms/src/IO/readFloatText.h index 7dea367d924..01303886b7e 100644 --- a/dbms/src/IO/readFloatText.h +++ b/dbms/src/IO/readFloatText.h @@ -312,7 +312,7 @@ template ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) { static_assert(std::is_same_v || std::is_same_v, "Argument for readFloatTextImpl must be float or double"); - static_assert('a' > '.' && 'A' > '.' && '\n' < '.' && '\t' < '.' && '\'' < '.' && '"' < '.', "Layout of char is not like ASCII"); + static_assert('a' > '.' && 'A' > '.' && '\n' < '.' && '\t' < '.' && '\'' < '.' && '"' < '.', "Layout of char is not like ASCII"); //-V501 static constexpr bool throw_exception = std::is_same_v; diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 5a13599bf89..c3f17f37b03 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -407,12 +407,12 @@ struct AggregatedDataVariants : private boost::noncopyable */ Aggregator * aggregator = nullptr; - size_t keys_size; /// Number of keys. NOTE do we need this field? + size_t keys_size{}; /// Number of keys. NOTE do we need this field? Sizes key_sizes; /// Dimensions of keys, if keys of fixed length /// Pools for states of aggregate functions. Ownership will be later transferred to ColumnAggregateFunction. Arenas aggregates_pools; - Arena * aggregates_pool; /// The pool that is currently used for allocation. + Arena * aggregates_pool{}; /// The pool that is currently used for allocation. /** Specialization for the case when there are no keys, and for keys not fitted into max_rows_to_group_by. */ diff --git a/dbms/src/Interpreters/Cluster.h b/dbms/src/Interpreters/Cluster.h index 8179caf10a0..06714da5cef 100644 --- a/dbms/src/Interpreters/Cluster.h +++ b/dbms/src/Interpreters/Cluster.h @@ -112,8 +112,8 @@ public: /// Name of directory for asynchronous write to StorageDistributed if has_internal_replication std::string dir_name_for_internal_replication; /// Number of the shard, the indexation begins with 1 - UInt32 shard_num; - UInt32 weight; + UInt32 shard_num = 0; + UInt32 weight = 1; Addresses local_addresses; /// nullptr if there are no remote addresses ConnectionPoolWithFailoverPtr pool; diff --git a/dbms/src/Interpreters/ExpressionJIT.cpp b/dbms/src/Interpreters/ExpressionJIT.cpp index 8cb9f2003e1..567341c2490 100644 --- a/dbms/src/Interpreters/ExpressionJIT.cpp +++ b/dbms/src/Interpreters/ExpressionJIT.cpp @@ -77,7 +77,7 @@ namespace { const char * data = nullptr; const char * null = nullptr; - size_t stride; + size_t stride = 0; }; struct ColumnDataPlaceholder diff --git a/dbms/src/Storages/MergeTree/BoolMask.h b/dbms/src/Storages/MergeTree/BoolMask.h index a7e66a917b0..75f9a6ff363 100644 --- a/dbms/src/Storages/MergeTree/BoolMask.h +++ b/dbms/src/Storages/MergeTree/BoolMask.h @@ -3,8 +3,8 @@ /// Multiple Boolean values. That is, two Boolean values: can it be true, can it be false. struct BoolMask { - bool can_be_true; - bool can_be_false; + bool can_be_true = false; + bool can_be_false = false; BoolMask() {} BoolMask(bool can_be_true_, bool can_be_false_) : can_be_true(can_be_true_), can_be_false(can_be_false_) {} diff --git a/libs/libmysqlxx/include/mysqlxx/Row.h b/libs/libmysqlxx/include/mysqlxx/Row.h index 6e22695bd19..a0b88638546 100644 --- a/libs/libmysqlxx/include/mysqlxx/Row.h +++ b/libs/libmysqlxx/include/mysqlxx/Row.h @@ -80,9 +80,9 @@ public: operator private_bool_type() const { return row == nullptr ? nullptr : &Row::row; } private: - MYSQL_ROW row = nullptr; - ResultBase * res = nullptr; - MYSQL_LENGTHS lengths; + MYSQL_ROW row{}; + ResultBase * res{}; + MYSQL_LENGTHS lengths{}; }; } diff --git a/libs/libmysqlxx/src/Query.cpp b/libs/libmysqlxx/src/Query.cpp index 57609e8365a..08474b8d864 100644 --- a/libs/libmysqlxx/src/Query.cpp +++ b/libs/libmysqlxx/src/Query.cpp @@ -40,6 +40,9 @@ Query::Query(const Query & other) : std::ostream(0), conn(other.conn) Query & Query::operator= (const Query & other) { + if (this == &other) + return *this; + conn = other.conn; seekp(0); diff --git a/utils/test-data-generator/CMakeLists.txt b/utils/test-data-generator/CMakeLists.txt index 787d29e4087..2e11b537873 100644 --- a/utils/test-data-generator/CMakeLists.txt +++ b/utils/test-data-generator/CMakeLists.txt @@ -1,9 +1,3 @@ -add_executable (test-data-generator main.cpp) -target_link_libraries(test-data-generator PRIVATE clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) - -add_executable (markov-model markov-model.cpp) -target_link_libraries(markov-model PRIVATE clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) - if(USE_PROTOBUF) protobuf_generate_cpp(ProtobufDelimitedMessagesSerializer_Srcs ProtobufDelimitedMessagesSerializer_Hdrs ${CMAKE_CURRENT_SOURCE_DIR}/../../dbms/tests/queries/0_stateless/00825_protobuf_format.proto) protobuf_generate_cpp(ProtobufDelimitedMessagesSerializer_Srcs2 ProtobufDelimitedMessagesSerializer_Hdrs2 ${CMAKE_CURRENT_SOURCE_DIR}/../../dbms/tests/queries/0_stateless/00825_protobuf_format_syntax2.proto) diff --git a/utils/test-data-generator/main.cpp b/utils/test-data-generator/main.cpp deleted file mode 100644 index 2a0db1e4b3d..00000000000 --- a/utils/test-data-generator/main.cpp +++ /dev/null @@ -1,378 +0,0 @@ -#include -#include -#include -#include - -#include -#include -#include -#include - -#include "MarkovModel.h" - - -/** Generate infinite stream of pseudo-random data - * like 'hits' table in Yandex.Metrica (with structure as it was in 2013) - * and output it in TabSeparated format. - */ - -using namespace DB; - - -struct Models -{ - MarkovModel Title; - MarkovModel URL; - MarkovModel SearchPhrase; - - static void read(MarkovModel & model, const String & path) - { - ReadBufferFromFile in(path); - model.read(in); - } - - Models() - { - read(Title, "Title.model"); - read(URL, "URL.model"); - read(SearchPhrase, "SearchPhrase.model"); - } -}; - - -struct Generator -{ - WriteBufferFromFileDescriptor out; - pcg64 random; - pcg64 random_with_seed; - Models models; - -// UInt64 WatchID = random(); - String Title; - String URL; -/* String Referer; - String FlashMinor2; - String UserAgentMinor; - String MobilePhoneModel; - String Params;*/ - String SearchPhrase; -/* String PageCharset; - String OriginalURL; - String BrowserLanguage; - String BrowserCountry; - String SocialNetwork; - String SocialAction; - String SocialSourcePage; - String ParamCurrency; - String OpenstatServiceName; - String OpenstatCampaignID; - String OpenstatAdID; - String OpenstatSourceID; - String UTMSource; - String UTMMedium; - String UTMCampaign; - String UTMContent; - String UTMTerm; - String FromTag;*/ - - Generator() : out(STDOUT_FILENO) {} - - /** Choosing of distributions parameters sometimes resembles real data, but quite arbitary. - */ - - void generateRow() - { -// auto gen_random64 = [&]{ return random(); }; - - /// Unique identifier of event. -/* WatchID += std::uniform_int_distribution(0, 10000000000)(random); - writeText(WatchID, out); - writeChar('\t', out); - - bool JavaEnable = std::bernoulli_distribution(0.6)(random); - writeText(JavaEnable, out); - writeChar('\t', out);*/ - - LocalDateTime EventTime; - EventTime.year(2013); - EventTime.month(7); - EventTime.day(std::discrete_distribution<>({ - 0, 0, 13, 30, 0, 14, 42, 5, 6, 31, 17, 0, 0, 0, 0, 23, 10, 0, 0, 0, 19, 24, 8, 7, 0, 0, 8, 2, 15, 12, 7, 29})(random)); - EventTime.hour(std::discrete_distribution<>({ - 13, 7, 4, 3, 2, 3, 4, 6, 10, 16, 20, 23, 24, 23, 18, 19, 19, 19, 14, 15, 14, 13, 17, 17})(random)); - EventTime.minute(std::uniform_int_distribution(0, 59)(random)); - EventTime.second(std::uniform_int_distribution(0, 59)(random)); - - UInt64 UserID = hash(4, powerLaw(5000, 1.1)); - UserID = UserID / 10000000000ULL * 10000000000ULL + static_cast(EventTime) + UserID % 1000000; - - random_with_seed.seed(powerLaw(5000, 1.1)); - auto get_random_with_seed = [&]{ return random_with_seed(); }; - - Title.resize(10000); - Title.resize(models.Title.generate(&Title[0], Title.size(), get_random_with_seed)); - writeText(Title, out); - writeChar('\t', out); - -/* bool GoodEvent = 1; - writeText(GoodEvent, out); - writeChar('\t', out);*/ - - writeText(EventTime, out); - writeChar('\t', out); - - LocalDate EventDate = EventTime.toDate(); - writeText(EventDate, out); - writeChar('\t', out); - - UInt32 CounterID = hash(1, powerLaw(20, 1.1)) % 10000000; - writeText(CounterID, out); - writeChar('\t', out); - -/* UInt32 ClientIP = hash(2, powerLaw(5000, 1.1)); - writeText(ClientIP, out); - writeChar('\t', out); - - UInt32 RegionID = hash(3, powerLaw(15, 1.1)) % 5000; - writeText(RegionID, out); - writeChar('\t', out); -*/ - writeText(UserID, out); - writeChar('\t', out); - -/* bool CounterClass = (hash(5, CounterID) % 100) < 25; - writeText(CounterClass, out); - writeChar('\t', out); - - UInt8 OS = hash(6, powerLaw(10, 4)) % 100; - writeText(OS, out); - writeChar('\t', out); - - UInt8 UserAgent = hash(7, powerLaw(10, 4)) % 100; - writeText(UserAgent, out); - writeChar('\t', out); -*/ - URL.resize(10000); - URL.resize(models.URL.generate(&URL[0], URL.size(), get_random_with_seed)); - writeText(URL, out); - writeChar('\t', out); - - /// Referer - -/* bool Refresh = std::bernoulli_distribution(0.1)(random); - writeText(Refresh, out); - writeChar('\t', out); - - UInt16 RefererCategoryID = std::bernoulli_distribution(0.1)(random) ? 0 : (hash(8, powerLaw(10, 4)) % 10000); - writeText(RefererCategoryID, out); - writeChar('\t', out); - - UInt32 RefererRegionID = std::bernoulli_distribution(0.1)(random) ? 0 : (hash(9, powerLaw(15, 1.1)) % 5000); - writeText(RefererRegionID, out); - writeChar('\t', out); - - UInt16 URLCategoryID = std::bernoulli_distribution(0.1)(random) ? 0 : (hash(10, powerLaw(10, 4)) % 10000); - writeText(URLCategoryID, out); - writeChar('\t', out); - - UInt32 URLRegionID = std::bernoulli_distribution(0.1)(random) ? 0 : (hash(11, powerLaw(15, 1.1)) % 5000); - writeText(URLRegionID, out); - writeChar('\t', out); - - UInt16 ResolutionWidth; - UInt16 ResolutionHeight; - - std::tie(ResolutionWidth, ResolutionHeight) = powerLawSampleFrom>(15, 1.1, - { - {1366, 768}, {1280, 1024}, {1920, 1080}, {0, 0}, {1024, 768}, - {1280, 800}, {1440, 900}, {1600, 900}, {1600, 900}, {1680, 1050}, - {768, 1024}, {1024, 600}, {1360, 768}, {1280, 720}, {1152, 864}, - {1280, 768}, {320, 480}, {1920, 1200}, {320, 568}, {1093, 614}, - }); - - if (std::bernoulli_distribution(0.1)(random)) - { - ResolutionWidth = std::bernoulli_distribution(0.1)(random) - ? std::uniform_int_distribution(160, 3000)(random) - : (std::uniform_int_distribution(160, 3000)(random) / 16 * 16); - - ResolutionHeight = std::bernoulli_distribution(0.1)(random) - ? std::uniform_int_distribution(160, 3000)(random) - : (ResolutionWidth / 16 * 10); - } - - writeText(ResolutionWidth, out); - writeChar('\t', out); - - writeText(ResolutionHeight, out); - writeChar('\t', out); - - UInt8 ResolutionDepth = weightedSelect({32, 24, 0, 16, 8}, {2000000, 700000, 300000, 50000, 100}); - writeText(ResolutionDepth, out); - writeChar('\t', out); - - UInt8 FlashMajor = weightedSelect({11, 0, 10, 6, 9, 8, 7, 5, 12}, {2000000, 600000, 200000, 100000, 8000, 800, 600, 20, 10}); - UInt8 FlashMinor = weightedSelect({7, 0, 8, 1, 6, 3, 2, 5, 4}, {1500000, 700000, 500000, 150000, 100000, 80000, 60000, 50000, 50000}); - - writeText(FlashMajor, out); - writeChar('\t', out); - - writeText(FlashMinor, out); - writeChar('\t', out); - - FlashMinor2.clear(); - if (FlashMajor && FlashMinor) - { - FlashMinor2 = toString(hash(12, powerLaw(10, 4)) % 1000); - if (std::bernoulli_distribution(0.2)(random)) - { - FlashMinor2 += '.'; - FlashMinor2 += toString(hash(13, powerLaw(10, 4)) % 1000); - } - } - - writeText(FlashMinor2, out); - writeChar('\t', out); - - UInt8 NetMajor = weightedSelect({0, 3, 2, 1, 4}, {3000000, 100000, 10000, 5000, 2}); - UInt8 NetMinor = weightedSelect({0, 5, 1}, {3000000, 200000, 5000}); - - writeText(NetMajor, out); - writeChar('\t', out); - - writeText(NetMinor, out); - writeChar('\t', out); - - UInt16 UserAgentMajor = UserAgent ? hash(14, powerLaw(10, 4)) % 100 : 0; - writeText(UserAgentMajor, out); - writeChar('\t', out); - - UserAgentMinor.clear(); - if (UserAgentMajor) - { - UserAgentMinor = toString(hash(15, powerLaw(10, 4)) % 100); - if (UserAgentMinor.size() == 1 && std::bernoulli_distribution(0.1)(random)) - UserAgentMinor += 'a' + std::uniform_int_distribution(0, 25)(random); - } - writeText(UserAgentMinor, out); - writeChar('\t', out); - - bool CookieEnable = std::bernoulli_distribution(0.999)(random); - writeText(CookieEnable, out); - writeChar('\t', out); - - bool JavascriptEnable = std::bernoulli_distribution(0.95)(random); - writeText(JavascriptEnable, out); - writeChar('\t', out); - - bool IsMobile = std::bernoulli_distribution(0.15)(random); - writeText(IsMobile, out); - writeChar('\t', out); - - UInt8 MobilePhone = IsMobile ? hash(16, powerLaw(10, 4)) % 100 : 0; - writeText(MobilePhone, out); - writeChar('\t', out); -*/ -/* MobilePhoneModel.resize(100); - MobilePhoneModel.resize(models.MobilePhoneModel.generate(&MobilePhoneModel[0], MobilePhoneModel.size(), gen_random64)); - writeText(MobilePhoneModel, out); - writeChar('\t', out); - - Params.resize(10000); - Params.resize(models.Params.generate(&Params[0], Params.size(), gen_random64)); - writeText(Params, out); - writeChar('\t', out); - - UInt32 IPNetworkID = hash(17, powerLaw(15, 1.1)) % 5000; - writeText(IPNetworkID, out); - writeChar('\t', out); - - Int8 TraficSourceID = weightedSelect( - {-1, 0, 1, 2, 3, 4, 5, 6, 7, 8}, {2000000, 300000, 200000, 600000, 50000, 700, 30000, 40000, 500, 2500}); - writeText(TraficSourceID, out); - writeChar('\t', out); - - UInt16 SearchEngineID = TraficSourceID == 2 - ? hash(18, powerLaw(10, 4)) % 100 - : (TraficSourceID == 3 - ? (std::bernoulli_distribution(0.5)(random) - ? hash(19, powerLaw(10, 4)) % 10 - : 0) - : 0); - - if (!SearchEngineID) - SearchPhrase.clear(); - else - {*/ - SearchPhrase.resize(1000); - SearchPhrase.resize(models.SearchPhrase.generate(&SearchPhrase[0], SearchPhrase.size(), get_random_with_seed)); -// } - writeText(SearchPhrase, out); - /* writeChar('\t', out); - - UInt8 AdvEngineID = weightedSelect( - {0, 2, 12, 17, 18, 27, 34, 36}, {3000000, 30000, 3000, 30000, 1, 100, 40, 30}); - writeText(AdvEngineID, out); - writeChar('\t', out); - - bool IsArtificial = std::bernoulli_distribution(0.07)(random); - writeText(IsArtificial, out); - writeChar('\t', out); - - UInt16 WindowClientWidth = std::max(3000, ResolutionWidth - hash(20, UserID) % 100); - UInt16 WindowClientHeight = std::max(3000, ResolutionHeight - hash(21, UserID) % 100); - - writeText(WindowClientWidth, out); - writeChar('\t', out); - - writeText(WindowClientHeight, out);*/ - writeChar('\n', out); - } - - UInt64 powerLawImpl(double scale, double alpha, double unit_random_value) - { - return scale * std::pow(unit_random_value, -1.0 / alpha) - scale; - } - - UInt64 powerLaw(double scale, double alpha) - { - return powerLawImpl(scale, alpha, std::uniform_real_distribution(0, 1)(random)); - } - - template - T powerLawSampleFrom(double scale, double alpha, std::initializer_list set) - { - return set.begin()[powerLaw(scale, alpha) % set.size()]; - } - - template - T weightedSelect(std::initializer_list items, std::initializer_list weights) - { - return items.begin()[std::discrete_distribution<>(weights)(random)]; - } - - static UInt64 hash(unsigned seed, UInt64 x) - { - return intHash64(x + seed * 0xDEADBEEF); - } -}; - - -int main(int argc, char ** argv) -try -{ - Generator generator; - while (true) - generator.generateRow(); - - return 0; -} -catch (...) -{ - /// Broken pipe, when piped to 'head', by example. - if (errno != EPIPE) - { - std::cerr << getCurrentExceptionMessage(true) << '\n'; - throw; - } -} diff --git a/utils/test-data-generator/markov-model.cpp b/utils/test-data-generator/markov-model.cpp deleted file mode 100644 index 87849f8f398..00000000000 --- a/utils/test-data-generator/markov-model.cpp +++ /dev/null @@ -1,114 +0,0 @@ -#include -#include -#include - -#include - -#include -#include -#include - -#include "MarkovModel.h" - -using namespace DB; - - -int main(int argc, char ** argv) -try -{ - boost::program_options::options_description desc("Allowed options"); - desc.add_options() - ("help,h", "produce help message") - ("create", "create model") - ("order", boost::program_options::value(), "order of model to create") - ("noise", boost::program_options::value(), "relative random noise to apply to created model") - ("generate", "generate random strings with model") - ("max-string-size", boost::program_options::value()->default_value(10000), "maximum size of generated string") - ("limit", boost::program_options::value(), "stop after specified count of generated strings") - ("seed", boost::program_options::value(), "seed passed to random number generator") - ; - - boost::program_options::variables_map options; - boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); - - auto show_usage = [&] - { - std::cout << "Usage: \n" - << argv[0] << " --create --order=N < strings.tsv > model\n" - << argv[0] << " --generate < model > strings.tsv\n\n"; - std::cout << desc << std::endl; - }; - - if (options.count("help")) - { - show_usage(); - return 1; - } - - ReadBufferFromFileDescriptor in(STDIN_FILENO); - WriteBufferFromFileDescriptor out(STDOUT_FILENO); - - pcg64 random; - - if (options.count("seed")) - random.seed(options["seed"].as()); - - if (options.count("create")) - { - MarkovModel model(options["order"].as()); - - String s; - while (!in.eof()) - { - readText(s, in); - assertChar('\n', in); - - model.consume(s.data(), s.size()); - } - - if (options.count("noise")) - { - double noise = options["noise"].as(); - model.modifyCounts([&](UInt32 count) - { - double modified = std::normal_distribution(count, count * noise)(random); - if (modified < 1) - modified = 1; - - return std::round(modified); - }); - } - - model.write(out); - } - else if (options.count("generate")) - { - MarkovModel model; - model.read(in); - String s; - - UInt64 limit = options.count("limit") ? options["limit"].as() : 0; - UInt64 max_string_size = options["max-string-size"].as(); - - for (size_t i = 0; limit == 0 || i < limit; ++i) - { - s.resize(max_string_size); - s.resize(model.generate(&s[0], s.size(), [&]{ return random(); })); - - writeText(s, out); - writeChar('\n', out); - } - } - else - { - show_usage(); - return 1; - } - - return 0; -} -catch (...) -{ - std::cerr << getCurrentExceptionMessage(true) << '\n'; - throw; -} From 58b26b427925413ed0d39f0486065a2cd1320ee0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Apr 2019 19:10:49 +0300 Subject: [PATCH 25/37] Removed obsolete scripts --- dbms/scripts/gen-bias-data.py | 262 --------- dbms/scripts/gen_benchmark_data/README.md | 1 - dbms/scripts/gen_benchmark_data/generate.py | 22 - dbms/scripts/gen_benchmark_data/model.py | 147 ----- .../gen_benchmark_data/requirements.txt | 3 - dbms/scripts/gen_benchmark_data/tokens | 506 ------------------ dbms/scripts/gen_benchmark_data/train.py | 26 - dbms/scripts/linear-counting-threshold.py | 150 ------ dbms/scripts/merge_algorithm/add_parts.sh | 10 - dbms/scripts/merge_algorithm/drawer.py | 76 --- dbms/scripts/merge_algorithm/stats.py | 61 --- .../test_intHash32_for_linear_counting.py | 56 -- dbms/scripts/test_uniq_functions.sh | 11 - 13 files changed, 1331 deletions(-) delete mode 100755 dbms/scripts/gen-bias-data.py delete mode 100644 dbms/scripts/gen_benchmark_data/README.md delete mode 100644 dbms/scripts/gen_benchmark_data/generate.py delete mode 100644 dbms/scripts/gen_benchmark_data/model.py delete mode 100644 dbms/scripts/gen_benchmark_data/requirements.txt delete mode 100644 dbms/scripts/gen_benchmark_data/tokens delete mode 100644 dbms/scripts/gen_benchmark_data/train.py delete mode 100755 dbms/scripts/linear-counting-threshold.py delete mode 100644 dbms/scripts/merge_algorithm/add_parts.sh delete mode 100644 dbms/scripts/merge_algorithm/drawer.py delete mode 100644 dbms/scripts/merge_algorithm/stats.py delete mode 100755 dbms/scripts/test_intHash32_for_linear_counting.py delete mode 100755 dbms/scripts/test_uniq_functions.sh diff --git a/dbms/scripts/gen-bias-data.py b/dbms/scripts/gen-bias-data.py deleted file mode 100755 index 034cfcca7dd..00000000000 --- a/dbms/scripts/gen-bias-data.py +++ /dev/null @@ -1,262 +0,0 @@ -#!/usr/bin/python3 -# -*- coding: utf-8 -*- - -import sys -import argparse -import tempfile -import random -import subprocess -import bisect -from copy import deepcopy - -# Псевдослучайный генератор уникальных чисел. -# http://preshing.com/20121224/how-to-generate-a-sequence-of-unique-random-integers/ -class UniqueRandomGenerator: - prime = 4294967291 - - def __init__(self, seed_base, seed_offset): - self.index = self.permutePQR(self.permutePQR(seed_base) + 0x682f0161) - self.intermediate_offset = self.permutePQR(self.permutePQR(seed_offset) + 0x46790905) - - def next(self): - val = self.permutePQR((self.permutePQR(self.index) + self.intermediate_offset) ^ 0x5bf03635) - self.index = self.index + 1 - return val - - def permutePQR(self, x): - if x >=self.prime: - return x - else: - residue = (x * x) % self.prime - if x <= self.prime/2: - return residue - else: - return self.prime - residue - -# Создать таблицу содержащую уникальные значения. -def generate_data_source(host, port, http_port, min_cardinality, max_cardinality, count): - chunk_size = round((max_cardinality - min_cardinality) / float(count)) - used_values = 0 - - cur_count = 0 - next_size = 0 - - sup = 32768 - n1 = random.randrange(0, sup) - n2 = random.randrange(0, sup) - urng = UniqueRandomGenerator(n1, n2) - - is_first = True - - with tempfile.TemporaryDirectory() as tmp_dir: - filename = tmp_dir + '/table.txt' - with open(filename, 'w+b') as file_handle: - while cur_count < count: - - if is_first == True: - is_first = False - if min_cardinality != 0: - next_size = min_cardinality + 1 - else: - next_size = chunk_size - else: - next_size += chunk_size - - while used_values < next_size: - h = urng.next() - used_values = used_values + 1 - out = str(h) + "\t" + str(cur_count) + "\n"; - file_handle.write(bytes(out, 'UTF-8')); - cur_count = cur_count + 1 - - query = "DROP TABLE IF EXISTS data_source" - subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query]) - query = "CREATE TABLE data_source(UserID UInt64, KeyID UInt64) ENGINE=TinyLog" - subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query]) - - cat = subprocess.Popen(("cat", filename), stdout=subprocess.PIPE) - subprocess.check_output(("POST", "http://{0}:{1}/?query=INSERT INTO data_source FORMAT TabSeparated".format(host, http_port)), stdin=cat.stdout) - cat.wait() - -def perform_query(host, port): - query = "SELECT runningAccumulate(uniqExactState(UserID)) AS exact, " - query += "runningAccumulate(uniqCombinedRawState(UserID)) AS approx " - query += "FROM data_source GROUP BY KeyID" - return subprocess.check_output(["clickhouse-client", "--host", host, "--port", port, "--query", query]) - -def parse_clickhouse_response(response): - parsed = [] - lines = response.decode().split("\n") - for cur_line in lines: - rows = cur_line.split("\t") - if len(rows) == 2: - parsed.append([float(rows[0]), float(rows[1])]) - return parsed - -def accumulate_data(accumulated_data, data): - if not accumulated_data: - accumulated_data = deepcopy(data) - else: - for row1, row2 in zip(accumulated_data, data): - row1[1] += row2[1]; - return accumulated_data - -def generate_raw_result(accumulated_data, count): - expected_tab = [] - bias_tab = [] - for row in accumulated_data: - exact = row[0] - expected = row[1] / count - bias = expected - exact - - expected_tab.append(expected) - bias_tab.append(bias) - return [ expected_tab, bias_tab ] - -def generate_sample(raw_estimates, biases, n_samples): - result = [] - - min_card = raw_estimates[0] - max_card = raw_estimates[len(raw_estimates) - 1] - step = (max_card - min_card) / (n_samples - 1) - - for i in range(0, n_samples + 1): - x = min_card + i * step - j = bisect.bisect_left(raw_estimates, x) - - if j == len(raw_estimates): - result.append((raw_estimates[j - 1], biases[j - 1])) - elif raw_estimates[j] == x: - result.append((raw_estimates[j], biases[j])) - else: - # Найти 6 ближайших соседей. Вычислить среднее арифметическое. - - # 6 точек слева x [j-6 j-5 j-4 j-3 j-2 j-1] - - begin = max(j - 6, 0) - 1 - end = j - 1 - - T = [] - for k in range(end, begin, -1): - T.append(x - raw_estimates[k]) - - # 6 точек справа x [j j+1 j+2 j+3 j+4 j+5] - - begin = j - end = min(j + 5, len(raw_estimates) - 1) + 1 - - U = [] - for k in range(begin, end): - U.append(raw_estimates[k] - x) - - # Сливаем расстояния. - - V = [] - - lim = min(len(T), len(U)) - k1 = 0 - k2 = 0 - - while k1 < lim and k2 < lim: - if T[k1] == U[k2]: - V.append(j - k1 - 1) - V.append(j + k2) - k1 = k1 + 1 - k2 = k2 + 1 - elif T[k1] < U[k2]: - V.append(j - k1 - 1) - k1 = k1 + 1 - else: - V.append(j + k2) - k2 = k2 + 1 - - if k1 < len(T): - while k1 < len(T): - V.append(j - k1 - 1) - k1 = k1 + 1 - elif k2 < len(U): - while k2 < len(U): - V.append(j + k2) - k2 = k2 + 1 - - # Выбираем 6 ближайших точек. - # Вычисляем средние. - - begin = 0 - end = min(len(V), 6) - - sum = 0 - bias = 0 - for k in range(begin, end): - sum += raw_estimates[V[k]] - bias += biases[V[k]] - sum /= float(end) - bias /= float(end) - - result.append((sum, bias)) - - # Пропустить последовательные результаты, чьи оценки одинаковые. - final_result = [] - last = -1 - for entry in result: - if entry[0] != last: - final_result.append((entry[0], entry[1])) - last = entry[0] - - return final_result - -def dump_arrays(data): - - print("Size of each array: {0}\n".format(len(data))) - - is_first = True - sep = '' - - print("raw_estimates = ") - print("{") - for row in data: - print("\t{0}{1}".format(sep, row[0])) - if is_first == True: - is_first = False - sep = "," - print("};") - - is_first = True - sep = "" - - print("\nbiases = ") - print("{") - for row in data: - print("\t{0}{1}".format(sep, row[1])) - if is_first == True: - is_first = False - sep = "," - print("};") - -def start(): - parser = argparse.ArgumentParser(description = "Generate bias correction tables for HyperLogLog-based functions.") - parser.add_argument("-x", "--host", default="localhost", help="ClickHouse server host name"); - parser.add_argument("-p", "--port", type=int, default=9000, help="ClickHouse server TCP port"); - parser.add_argument("-t", "--http_port", type=int, default=8123, help="ClickHouse server HTTP port"); - parser.add_argument("-i", "--iterations", type=int, default=5000, help="number of iterations"); - parser.add_argument("-m", "--min_cardinality", type=int, default=16384, help="minimal cardinality"); - parser.add_argument("-M", "--max_cardinality", type=int, default=655360, help="maximal cardinality"); - parser.add_argument("-s", "--samples", type=int, default=200, help="number of sampled values"); - args = parser.parse_args() - - accumulated_data = [] - - for i in range(0, args.iterations): - print(i + 1) - sys.stdout.flush() - - generate_data_source(args.host, str(args.port), str(args.http_port), args.min_cardinality, args.max_cardinality, 1000) - response = perform_query(args.host, str(args.port)) - data = parse_clickhouse_response(response) - accumulated_data = accumulate_data(accumulated_data, data) - - result = generate_raw_result(accumulated_data, args.iterations) - sampled_data = generate_sample(result[0], result[1], args.samples) - dump_arrays(sampled_data) - -if __name__ == "__main__": start() diff --git a/dbms/scripts/gen_benchmark_data/README.md b/dbms/scripts/gen_benchmark_data/README.md deleted file mode 100644 index c7f8bd30391..00000000000 --- a/dbms/scripts/gen_benchmark_data/README.md +++ /dev/null @@ -1 +0,0 @@ -Hits table generator based on LSTM neural network trained on real hits. You need to have weights for model or train model on real hits to generate data. diff --git a/dbms/scripts/gen_benchmark_data/generate.py b/dbms/scripts/gen_benchmark_data/generate.py deleted file mode 100644 index b54651fe1b1..00000000000 --- a/dbms/scripts/gen_benchmark_data/generate.py +++ /dev/null @@ -1,22 +0,0 @@ -import argparse - -from model import Model -parser = argparse.ArgumentParser( - formatter_class=argparse.ArgumentDefaultsHelpFormatter) -parser.add_argument('-n', type=int, default=100000, - help='number of objects to generate') -parser.add_argument('--output_file', type=str, default='out.tsv', - help='output file name') -parser.add_argument('--weights_path', type=str, - help='path to weights') - - -args = parser.parse_args() - -if __name__ == '__main__': - if not args.weights_path: - raise Exception('please specify path to model weights with --weights_path') - - gen = Model() - gen.generate(args.n, args.output_file, args.weights_path) - diff --git a/dbms/scripts/gen_benchmark_data/model.py b/dbms/scripts/gen_benchmark_data/model.py deleted file mode 100644 index 3e2ec9c4942..00000000000 --- a/dbms/scripts/gen_benchmark_data/model.py +++ /dev/null @@ -1,147 +0,0 @@ -import numpy as np -import os -import pickle -import tensorflow as tf - -from random import sample -from keras.layers import Dense, Embedding -from tqdm import tqdm - -RNN_NUM_UNITS = 256 -EMB_SIZE = 32 -MAX_LENGTH = 1049 - - -with open('tokens', 'rb') as f: - tokens = pickle.load(f) -n_tokens = len(tokens) - -token_to_id = {c: i for i, c in enumerate(tokens)} - - -def to_matrix(objects, max_len=None, pad=0, dtype='int32'): - max_len = max_len or max(map(len, objects)) - matrix = np.zeros([len(objects), max_len], dtype) + pad - - for i in range(len(objects)): - name_ix = list(map(token_to_id.get, objects[i])) - matrix[i, :len(name_ix)] = name_ix - return matrix.T - - -class Model: - def __init__(self, learning_rate=0.0001): - # an embedding layer that converts character ids into embeddings - self.embed_x = Embedding(n_tokens, EMB_SIZE) - get_h_next = Dense(1024, activation='relu') - # a dense layer that maps current hidden state - # to probabilities of characters [h_t+1]->P(x_t+1|h_t+1) - self.get_probas = Dense(n_tokens, activation='softmax') - - self.input_sequence = tf.placeholder('int32', (MAX_LENGTH, None)) - batch_size = tf.shape(self.input_sequence)[1] - - self.gru_cell_first = tf.nn.rnn_cell.GRUCell(RNN_NUM_UNITS) - self.lstm_cell_second = tf.nn.rnn_cell.LSTMCell(RNN_NUM_UNITS) - - h_prev_first = self.gru_cell_first.zero_state(batch_size, dtype=tf.float32) - h_prev_second = tf.nn.rnn_cell.LSTMStateTuple( - tf.zeros([batch_size, RNN_NUM_UNITS]), # initial cell state, - tf.zeros([batch_size, RNN_NUM_UNITS]) # initial hidden state - ) - - predicted_probas = [] - for t in range(MAX_LENGTH): - x_t = self.input_sequence[t] - # convert character id into embedding - x_t_emb = self.embed_x(tf.reshape(x_t, [-1, 1]))[:, 0] - - out_next_first, h_next_first = self.gru_cell_first(x_t_emb, h_prev_first) - h_prev_first = h_next_first - - out_next_second, h_next_second = self.lstm_cell_second(out_next_first, h_prev_second) - h_prev_second = h_next_second - - probas_next = self.get_probas(out_next_second) - predicted_probas.append(probas_next) - - predicted_probas = tf.stack(predicted_probas) - - predictions_matrix = tf.reshape(predicted_probas[:-1], [-1, len(tokens)]) - answers_matrix = tf.one_hot(tf.reshape(self.input_sequence[1:], [-1]), n_tokens) - - self.loss = tf.reduce_mean(tf.reduce_sum( - -answers_matrix * tf.log(tf.clip_by_value(predictions_matrix, 1e-7, 1.0)), - reduction_indices=[1] - )) - optimizer = tf.train.AdamOptimizer(learning_rate) - gvs = optimizer.compute_gradients(self.loss) - capped_gvs = [(gr if gr is None else tf.clip_by_value(gr, -1., 1.), var) for gr, var in gvs] - self.optimize = optimizer.apply_gradients(capped_gvs) - - self.sess = tf.Session() - self.sess.run(tf.global_variables_initializer()) - self.saver = tf.train.Saver() - - def train(self, train_data_path, save_dir, num_iters, batch_size=64, restore_from=False): - history = [] - if restore_from: - with open(restore_from + '_history') as f: - history = pickle.load(f) - self.saver.restore(self.sess, restore_from) - with open(train_data_path, 'r') as f: - train_data = f.readlines() - - train_data = filter(lambda a: len(a) < MAX_LENGTH, train_data) - - for i in tqdm(range(num_iters)): - batch = to_matrix( - map(lambda a: '\n' + a.rstrip('\n'), sample(train_data, batch_size)), - max_len=MAX_LENGTH - ) - loss_i, _ = self.sess.run([self.loss, self.optimize], {self.input_sequence: batch}) - history.append(loss_i) - if len(history) % 2000 == 0: - self.saver.save(self.sess, os.path.join(save_dir, '{}_iters'.format(len(history)))) - self.saver.save(self.sess, os.path.join(save_dir, '{}_iters'.format(len(history)))) - with open(os.path.join(save_dir, '{}_iters_history'.format(len(history)))) as f: - pickle.dump(history, f) - - def generate(self, num_objects, output_file, weights_path): - self.saver.restore(self.sess, weights_path) - batch_size = num_objects - x_t = tf.placeholder('int32', (None, batch_size)) - h_t_first = tf.Variable(tf.zeros([batch_size, RNN_NUM_UNITS])) - h_t_second = tf.nn.rnn_cell.LSTMStateTuple( - tf.Variable(tf.zeros([batch_size, RNN_NUM_UNITS])), - tf.Variable(tf.zeros([batch_size, RNN_NUM_UNITS])) - ) - - x_t_emb = self.embed_x(tf.reshape(x_t, [-1, 1]))[:, 0] - first_out_next, next_h_first = self.gru_cell_first(x_t_emb, h_t_first) - second_out_next, next_h_second = self.lstm_cell_second(first_out_next, h_t_second) - next_probs = self.get_probas(second_out_next) - - x_sequence = np.zeros(shape=(1, batch_size), dtype=int) + token_to_id['\n'] - self.sess.run( - [tf.assign(h_t_first, h_t_first.initial_value), - tf.assign(h_t_second[0], h_t_second[0].initial_value), - tf.assign(h_t_second[1], h_t_second[1].initial_value)] - ) - - for i in tqdm(range(MAX_LENGTH - 1)): - x_probs, _, _, _ = self.sess.run( - [next_probs, - tf.assign(h_t_second[0], next_h_second[0]), - tf.assign(h_t_second[1], next_h_second[1]), - tf.assign(h_t_first, next_h_first)], - {x_t: [x_sequence[-1, :]]} - ) - - next_char = [np.random.choice(n_tokens, p=x_probs[i]) for i in range(batch_size)] - if sum(next_char) == 0: - break - x_sequence = np.append(x_sequence, [next_char], axis=0) - - with open(output_file, 'w') as f: - f.writelines([''.join([tokens[ix] for ix in x_sequence.T[k]]) + '\n' for k in range(batch_size)]) diff --git a/dbms/scripts/gen_benchmark_data/requirements.txt b/dbms/scripts/gen_benchmark_data/requirements.txt deleted file mode 100644 index b02bc51fee1..00000000000 --- a/dbms/scripts/gen_benchmark_data/requirements.txt +++ /dev/null @@ -1,3 +0,0 @@ -Keras==2.0.6 -numpy -tensorflow-gpu==1.4.0 \ No newline at end of file diff --git a/dbms/scripts/gen_benchmark_data/tokens b/dbms/scripts/gen_benchmark_data/tokens deleted file mode 100644 index f80b0dd4208..00000000000 --- a/dbms/scripts/gen_benchmark_data/tokens +++ /dev/null @@ -1,506 +0,0 @@ -(lp0 -S'\x83' -p1 -aS'\x04' -p2 -aS'\x87' -p3 -aS'\x8b' -p4 -aS'\x8f' -p5 -aS'\x10' -p6 -aS'\x93' -p7 -aS'\x14' -p8 -aS'\x97' -p9 -aS'\x18' -p10 -aS'\x9b' -p11 -aS'\x1c' -p12 -aS'\x9f' -p13 -aS' ' -p14 -aS'\xa3' -p15 -aS'$' -p16 -aS'\xa7' -p17 -aS'(' -p18 -aS'\xab' -p19 -aS',' -p20 -aS'\xaf' -p21 -aS'0' -p22 -aS'\xb3' -p23 -aS'4' -p24 -aS'\xb7' -p25 -aS'8' -p26 -aS'\xbb' -p27 -aS'<' -p28 -aS'\xbf' -p29 -aS'@' -p30 -aS'\xc3' -p31 -aS'D' -p32 -aS'\xc7' -p33 -aS'H' -p34 -aS'\xcb' -p35 -aS'L' -p36 -aS'\xcf' -p37 -aS'P' -p38 -aS'\xd3' -p39 -aS'T' -p40 -aS'\xd7' -p41 -aS'X' -p42 -aS'\xdb' -p43 -aS'\\' -p44 -aS'\xdf' -p45 -aS'`' -p46 -aS'\xe3' -p47 -aS'd' -p48 -aS'\xe7' -p49 -aS'h' -p50 -aS'\xeb' -p51 -aS'l' -p52 -aS'\xef' -p53 -aS'p' -p54 -aS'\xf3' -p55 -aS't' -p56 -aS'\xf7' -p57 -aS'x' -p58 -aS'\xfb' -p59 -aS'|' -p60 -aS'\xff' -p61 -aS'\x80' -p62 -aS'\x03' -p63 -aS'\x84' -p64 -aS'\x07' -p65 -aS'\x88' -p66 -aS'\x0b' -p67 -aS'\x8c' -p68 -aS'\x0f' -p69 -aS'\x90' -p70 -aS'\x13' -p71 -aS'\x94' -p72 -aS'\x17' -p73 -aS'\x98' -p74 -aS'\x1b' -p75 -aS'\x9c' -p76 -aS'\x1f' -p77 -aS'\xa0' -p78 -aS'#' -p79 -aS'\xa4' -p80 -aS"'" -p81 -aS'\xa8' -p82 -aS'+' -p83 -aS'\xac' -p84 -aS'/' -p85 -aS'\xb0' -p86 -aS'3' -p87 -aS'\xb4' -p88 -aS'7' -p89 -aS'\xb8' -p90 -aS';' -p91 -aS'\xbc' -p92 -aS'?' -p93 -aS'\xc0' -p94 -aS'C' -p95 -aS'\xc4' -p96 -aS'G' -p97 -aS'\xc8' -p98 -aS'K' -p99 -aS'\xcc' -p100 -aS'O' -p101 -aS'\xd0' -p102 -aS'S' -p103 -aS'\xd4' -p104 -aS'W' -p105 -aS'\xd8' -p106 -aS'[' -p107 -aS'\xdc' -p108 -aS'_' -p109 -aS'\xe0' -p110 -aS'c' -p111 -aS'\xe4' -p112 -aS'g' -p113 -aS'\xe8' -p114 -aS'k' -p115 -aS'\xec' -p116 -aS'o' -p117 -aS'\xf0' -p118 -aS's' -p119 -aS'\xf4' -p120 -aS'w' -p121 -aS'\xf8' -p122 -aS'{' -p123 -aS'\xfc' -p124 -aS'\x7f' -p125 -aS'\x81' -p126 -aS'\x02' -p127 -aS'\x85' -p128 -aS'\x06' -p129 -aS'\x89' -p130 -aS'\n' -p131 -aS'\x8d' -p132 -aS'\x0e' -p133 -aS'\x91' -p134 -aS'\x12' -p135 -aS'\x95' -p136 -aS'\x16' -p137 -aS'\x99' -p138 -aS'\x1a' -p139 -aS'\x9d' -p140 -aS'\x1e' -p141 -aS'\xa1' -p142 -aS'"' -p143 -aS'\xa5' -p144 -aS'&' -p145 -aS'\xa9' -p146 -aS'*' -p147 -aS'\xad' -p148 -aS'.' -p149 -aS'\xb1' -p150 -aS'2' -p151 -aS'\xb5' -p152 -aS'6' -p153 -aS'\xb9' -p154 -aS':' -p155 -aS'\xbd' -p156 -aS'>' -p157 -aS'\xc1' -p158 -aS'B' -p159 -aS'\xc5' -p160 -aS'F' -p161 -aS'\xc9' -p162 -aS'J' -p163 -aS'\xcd' -p164 -aS'N' -p165 -aS'\xd1' -p166 -aS'R' -p167 -aS'\xd5' -p168 -aS'V' -p169 -aS'\xd9' -p170 -aS'Z' -p171 -aS'\xdd' -p172 -aS'^' -p173 -aS'\xe1' -p174 -aS'b' -p175 -aS'\xe5' -p176 -aS'f' -p177 -aS'\xe9' -p178 -aS'j' -p179 -aS'\xed' -p180 -aS'n' -p181 -aS'\xf1' -p182 -aS'r' -p183 -aS'\xf5' -p184 -aS'v' -p185 -aS'\xf9' -p186 -aS'z' -p187 -aS'\xfd' -p188 -aS'~' -p189 -aS'\x01' -p190 -aS'\x82' -p191 -aS'\x05' -p192 -aS'\x86' -p193 -aS'\t' -p194 -aS'\x8a' -p195 -aS'\x8e' -p196 -aS'\x11' -p197 -aS'\x92' -p198 -aS'\x15' -p199 -aS'\x96' -p200 -aS'\x19' -p201 -aS'\x9a' -p202 -aS'\x1d' -p203 -aS'\x9e' -p204 -aS'!' -p205 -aS'\xa2' -p206 -aS'%' -p207 -aS'\xa6' -p208 -aS')' -p209 -aS'\xaa' -p210 -aS'-' -p211 -aS'\xae' -p212 -aS'1' -p213 -aS'\xb2' -p214 -aS'5' -p215 -aS'\xb6' -p216 -aS'9' -p217 -aS'\xba' -p218 -aS'=' -p219 -aS'\xbe' -p220 -aS'A' -p221 -aS'\xc2' -p222 -aS'E' -p223 -aS'\xc6' -p224 -aS'I' -p225 -aS'\xca' -p226 -aS'M' -p227 -aS'\xce' -p228 -aS'Q' -p229 -aS'\xd2' -p230 -aS'U' -p231 -aS'\xd6' -p232 -aS'Y' -p233 -aS'\xda' -p234 -aS']' -p235 -aS'\xde' -p236 -aS'a' -p237 -aS'\xe2' -p238 -aS'e' -p239 -aS'\xe6' -p240 -aS'i' -p241 -aS'\xea' -p242 -aS'm' -p243 -aS'\xee' -p244 -aS'q' -p245 -aS'\xf2' -p246 -aS'u' -p247 -aS'\xf6' -p248 -aS'y' -p249 -aS'\xfa' -p250 -aS'}' -p251 -aS'\xfe' -p252 -a. \ No newline at end of file diff --git a/dbms/scripts/gen_benchmark_data/train.py b/dbms/scripts/gen_benchmark_data/train.py deleted file mode 100644 index fd93805f50e..00000000000 --- a/dbms/scripts/gen_benchmark_data/train.py +++ /dev/null @@ -1,26 +0,0 @@ -import argparse - -from model import Model -parser = argparse.ArgumentParser( - formatter_class=argparse.ArgumentDefaultsHelpFormatter) -parser.add_argument('--n_iter', type=int, default=10000, - help='number of iterations') -parser.add_argument('--save_dir', type=str, default='save', - help='dir for saving weights') -parser.add_argument('--data_path', type=str, - help='path to train data') -parser.add_argument('--learning_rate', type=int, default=0.0001, - help='learning rate') -parser.add_argument('--batch_size', type=int, default=64, - help='batch size') -parser.add_argument('--restore_from', type=str, - help='path to train saved weights') - -args = parser.parse_args() - -if __name__ == '__main__': - if not args.data_path: - raise Exception('please specify path to train data with --data_path') - - gen = Model(args.learning_rate) - gen.train(args.data_path, args.save_dir, args.n_iter, args.batch_size, args.restore_from) diff --git a/dbms/scripts/linear-counting-threshold.py b/dbms/scripts/linear-counting-threshold.py deleted file mode 100755 index 9ed13cb4e4a..00000000000 --- a/dbms/scripts/linear-counting-threshold.py +++ /dev/null @@ -1,150 +0,0 @@ -#!/usr/bin/python3.4 -# -*- coding: utf-8 -*- - -import sys -import argparse -import tempfile -import random -import subprocess -import bisect -from copy import deepcopy - -# Псевдослучайный генератор уникальных чисел. -# http://preshing.com/20121224/how-to-generate-a-sequence-of-unique-random-integers/ -class UniqueRandomGenerator: - prime = 4294967291 - - def __init__(self, seed_base, seed_offset): - self.index = self.permutePQR(self.permutePQR(seed_base) + 0x682f0161) - self.intermediate_offset = self.permutePQR(self.permutePQR(seed_offset) + 0x46790905) - - def next(self): - val = self.permutePQR((self.permutePQR(self.index) + self.intermediate_offset) ^ 0x5bf03635) - self.index = self.index + 1 - return val - - def permutePQR(self, x): - if x >=self.prime: - return x - else: - residue = (x * x) % self.prime - if x <= self.prime/2: - return residue - else: - return self.prime - residue - -# Создать таблицу содержащую уникальные значения. -def generate_data_source(host, port, http_port, min_cardinality, max_cardinality, count): - chunk_size = round((max_cardinality - (min_cardinality + 1)) / float(count)) - used_values = 0 - - cur_count = 0 - next_size = 0 - - sup = 32768 - n1 = random.randrange(0, sup) - n2 = random.randrange(0, sup) - urng = UniqueRandomGenerator(n1, n2) - - is_first = True - - with tempfile.TemporaryDirectory() as tmp_dir: - filename = tmp_dir + '/table.txt' - with open(filename, 'w+b') as file_handle: - while cur_count < count: - - if is_first == True: - is_first = False - if min_cardinality != 0: - next_size = min_cardinality + 1 - else: - next_size = chunk_size - else: - next_size += chunk_size - - while used_values < next_size: - h = urng.next() - used_values = used_values + 1 - out = str(h) + "\t" + str(cur_count) + "\n"; - file_handle.write(bytes(out, 'UTF-8')); - cur_count = cur_count + 1 - - query = "DROP TABLE IF EXISTS data_source" - subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query]) - query = "CREATE TABLE data_source(UserID UInt64, KeyID UInt64) ENGINE=TinyLog" - subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query]) - - cat = subprocess.Popen(("cat", filename), stdout=subprocess.PIPE) - subprocess.check_output(("POST", "http://{0}:{1}/?query=INSERT INTO data_source FORMAT TabSeparated".format(host, http_port)), stdin=cat.stdout) - cat.wait() - -def perform_query(host, port): - query = "SELECT runningAccumulate(uniqExactState(UserID)) AS exact, " - query += "runningAccumulate(uniqCombinedRawState(UserID)) AS raw, " - query += "runningAccumulate(uniqCombinedLinearCountingState(UserID)) AS linear_counting, " - query += "runningAccumulate(uniqCombinedBiasCorrectedState(UserID)) AS bias_corrected " - query += "FROM data_source GROUP BY KeyID" - return subprocess.check_output(["clickhouse-client", "--host", host, "--port", port, "--query", query]) - -def parse_clickhouse_response(response): - parsed = [] - lines = response.decode().split("\n") - for cur_line in lines: - rows = cur_line.split("\t") - if len(rows) == 4: - parsed.append([float(rows[0]), float(rows[1]), float(rows[2]), float(rows[3])]) - return parsed - -def accumulate_data(accumulated_data, data): - if not accumulated_data: - accumulated_data = deepcopy(data) - else: - for row1, row2 in zip(accumulated_data, data): - row1[1] += row2[1]; - row1[2] += row2[2]; - row1[3] += row2[3]; - return accumulated_data - -def dump_graphs(data, count): - with open("raw_graph.txt", "w+b") as fh1, open("linear_counting_graph.txt", "w+b") as fh2, open("bias_corrected_graph.txt", "w+b") as fh3: - expected_tab = [] - bias_tab = [] - for row in data: - exact = row[0] - raw = row[1] / count; - linear_counting = row[2] / count; - bias_corrected = row[3] / count; - - outstr = "{0}\t{1}\n".format(exact, abs(raw - exact) / exact) - fh1.write(bytes(outstr, 'UTF-8')) - - outstr = "{0}\t{1}\n".format(exact, abs(linear_counting - exact) / exact) - fh2.write(bytes(outstr, 'UTF-8')) - - outstr = "{0}\t{1}\n".format(exact, abs(bias_corrected - exact) / exact) - fh3.write(bytes(outstr, 'UTF-8')) - -def start(): - parser = argparse.ArgumentParser(description = "Generate graphs that help to determine the linear counting threshold.") - parser.add_argument("-x", "--host", default="localhost", help="clickhouse host name"); - parser.add_argument("-p", "--port", type=int, default=9000, help="clickhouse client TCP port"); - parser.add_argument("-t", "--http_port", type=int, default=8123, help="clickhouse HTTP port"); - parser.add_argument("-i", "--iterations", type=int, default=5000, help="number of iterations"); - parser.add_argument("-m", "--min_cardinality", type=int, default=16384, help="minimal cardinality"); - parser.add_argument("-M", "--max_cardinality", type=int, default=655360, help="maximal cardinality"); - args = parser.parse_args() - - accumulated_data = [] - - for i in range(0, args.iterations): - print(i + 1) - sys.stdout.flush() - - generate_data_source(args.host, str(args.port), str(args.http_port), args.min_cardinality, args.max_cardinality, 1000) - response = perform_query(args.host, str(args.port)) - data = parse_clickhouse_response(response) - accumulated_data = accumulate_data(accumulated_data, data) - - dump_graphs(accumulated_data, args.iterations) - -if __name__ == "__main__": start() diff --git a/dbms/scripts/merge_algorithm/add_parts.sh b/dbms/scripts/merge_algorithm/add_parts.sh deleted file mode 100644 index cf1a5ee8fc2..00000000000 --- a/dbms/scripts/merge_algorithm/add_parts.sh +++ /dev/null @@ -1,10 +0,0 @@ -#!/usr/bin/env bash - -for (( i = 0; i < 1000; i++ )); do - if (( RANDOM % 10 )); then - clickhouse-client --port=9007 --query="INSERT INTO mt (x) SELECT rand64() AS x FROM system.numbers LIMIT 100000" - else - clickhouse-client --port=9007 --query="INSERT INTO mt (x) SELECT rand64() AS x FROM system.numbers LIMIT 300000" - fi - -done diff --git a/dbms/scripts/merge_algorithm/drawer.py b/dbms/scripts/merge_algorithm/drawer.py deleted file mode 100644 index aa1bf2e4d19..00000000000 --- a/dbms/scripts/merge_algorithm/drawer.py +++ /dev/null @@ -1,76 +0,0 @@ -from __future__ import print_function - -import argparse -import matplotlib.pyplot as plt -import ast - -TMP_FILE='tmp.tsv' - -def parse_args(): - parser = argparse.ArgumentParser(formatter_class=argparse.ArgumentDefaultsHelpFormatter) - parser.add_argument('-f', '--file', default='data.tsv') - cfg = parser.parse_args() - return cfg - -def draw(): - place = dict() - max_coord = 0 - global_top = 0 - for line in open(TMP_FILE): - numbers = line.split('\t') - if len(numbers) <= 2: - continue - name = numbers[-2] - if numbers[0] == '1': - dx = int(numbers[3]) - max_coord += dx - place[name] = [1, max_coord, 1, dx] - max_coord += dx - plt.plot([max_coord - 2 * dx, max_coord], [1, 1]) - for line in open(TMP_FILE): - numbers = line.split('\t') - if len(numbers) <= 2: - continue - name = numbers[-2] - if numbers[0] == '2': - list = ast.literal_eval(numbers[-1]) - coord = [0,0,0,0] - for cur_name in list: - coord[0] = max(place[cur_name][0], coord[0]) - coord[1] += place[cur_name][1] * place[cur_name][2] - coord[2] += place[cur_name][2] - coord[3] += place[cur_name][3] - coord[1] /= coord[2] - coord[0] += 1 - global_top = max(global_top, coord[0]) - place[name] = coord - for cur_name in list: - plt.plot([coord[1], place[cur_name][1]],[coord[0], place[cur_name][0]]) - plt.plot([coord[1] - coord[3], coord[1] + coord[3]], [coord[0], coord[0]]) - plt.plot([0], [global_top + 1]) - plt.plot([0], [-1]) - plt.show() - - -def convert(input_file): - print(input_file) - tmp_file = open(TMP_FILE, "w") - for line in open(input_file): - numbers = line.split('\t') - numbers2 = numbers[-2].split('_') - if numbers2[-2] == numbers2[-3]: - numbers2[-2] = str(int(numbers2[-2]) + 1) - numbers2[-3] = str(int(numbers2[-3]) + 1) - numbers[-2] = '_'.join(numbers2[1:]) - print('\t'.join(numbers), end='', file=tmp_file) - else: - print(line, end='', file=tmp_file) - -def main(): - cfg = parse_args() - convert(cfg.file) - draw() - -if __name__ == '__main__': - main() - diff --git a/dbms/scripts/merge_algorithm/stats.py b/dbms/scripts/merge_algorithm/stats.py deleted file mode 100644 index 52272b7ff88..00000000000 --- a/dbms/scripts/merge_algorithm/stats.py +++ /dev/null @@ -1,61 +0,0 @@ -import time -import ast -from datetime import datetime - -FILE='data.tsv' - -def get_metrix(): - data = [] - time_to_merge = 0 - count_of_parts = 0 - max_count_of_parts = 0 - parts_in_time = [] - last_date = 0 - for line in open(FILE): - fields = line.split('\t') - last_date = datetime.strptime(fields[2], '%Y-%m-%d %H:%M:%S') - break - - for line in open(FILE): - fields = line.split('\t') - cur_date = datetime.strptime(fields[2], '%Y-%m-%d %H:%M:%S') - if fields[0] == '2': - time_to_merge += int(fields[4]) - list = ast.literal_eval(fields[-1]) - count_of_parts -= len(list) - 1 - else: - count_of_parts += 1 - - if max_count_of_parts < count_of_parts: - max_count_of_parts = count_of_parts - - parts_in_time.append([(cur_date-last_date).total_seconds(), count_of_parts]) - last_date = cur_date - - stats_parts_in_time = [] - global_time = 0 - average_parts = 0 - for i in range(max_count_of_parts + 1): - stats_parts_in_time.append(0) - - for elem in parts_in_time: - stats_parts_in_time[elem[1]] += elem[0] - global_time += elem[0] - average_parts += elem[0] * elem[1] - - for i in range(max_count_of_parts): - stats_parts_in_time[i] /= global_time - average_parts /= global_time - - return time_to_merge, max_count_of_parts, average_parts, stats_parts_in_time - -def main(): - time_to_merge, max_parts, average_parts, stats_parts = get_metrix() - print('time_to_merge=', time_to_merge) - print('max_parts=', max_parts) - print('average_parts=', average_parts) - print('stats_parts=', stats_parts) - - -if __name__ == '__main__': - main() diff --git a/dbms/scripts/test_intHash32_for_linear_counting.py b/dbms/scripts/test_intHash32_for_linear_counting.py deleted file mode 100755 index 5a6900921cc..00000000000 --- a/dbms/scripts/test_intHash32_for_linear_counting.py +++ /dev/null @@ -1,56 +0,0 @@ -#!/usr/bin/python3 -import sys -import math -import statistics as stat - -start = int(sys.argv[1]) -end = int(sys.argv[2]) - -#Copied from dbms/src/Common/HashTable/Hash.h -def intHash32(key, salt = 0): - key ^= salt; - - key = (~key) + (key << 18); - key = key ^ ((key >> 31) | (key << 33)); - key = key * 21; - key = key ^ ((key >> 11) | (key << 53)); - key = key + (key << 6); - key = key ^ ((key >> 22) | (key << 42)); - - return key & 0xffffffff - -#Number of buckets for precision p = 12, m = 2^p -m = 4096 -n = start -c = 0 -m1 = {} -m2 = {} -l1 = [] -l2 = [] -while n <= end: - c += 1 - - h = intHash32(n) - #Extract left most 12 bits - x1 = (h >> 20) & 0xfff - m1[x1] = 1 - z1 = m - len(m1) - #Linear counting formula - u1 = int(m * math.log(float(m) / float(z1))) - e1 = abs(100*float(u1 - c)/float(c)) - l1.append(e1) - print("%d %d %d %f" % (n, c, u1, e1)) - - #Extract right most 12 bits - x2 = h & 0xfff - m2[x2] = 1 - z2 = m - len(m2) - u2 = int(m * math.log(float(m) / float(z2))) - e2 = abs(100*float(u2 - c)/float(c)) - l2.append(e2) - print("%d %d %d %f" % (n, c, u2, e2)) - - n += 1 - -print("Left 12 bits error: min=%f max=%f avg=%f median=%f median_low=%f median_high=%f" % (min(l1), max(l1), stat.mean(l1), stat.median(l1), stat.median_low(l1), stat.median_high(l1))) -print("Right 12 bits error: min=%f max=%f avg=%f median=%f median_low=%f median_high=%f" % (min(l2), max(l2), stat.mean(l2), stat.median(l2), stat.median_low(l2), stat.median_high(l2))) diff --git a/dbms/scripts/test_uniq_functions.sh b/dbms/scripts/test_uniq_functions.sh deleted file mode 100755 index 9a4b6f20433..00000000000 --- a/dbms/scripts/test_uniq_functions.sh +++ /dev/null @@ -1,11 +0,0 @@ -#!/usr/bin/env bash - -for ((p = 2; p <= 10; p++)) -do - for ((i = 1; i <= 9; i++)) - do - n=$(( 10**p * i )) - echo -n "$n " - clickhouse-client -q "select uniqHLL12(number), uniq(number), uniqCombined(number) from numbers($n);" - done -done From 720f9113799b9436bd048eb8a5edb2df9b1f4b8c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Apr 2019 22:28:10 +0300 Subject: [PATCH 26/37] Removed obsolete files --- utils/travis/normal.sh | 45 ---------------------------------------- utils/travis/pbuilder.sh | 35 ------------------------------- 2 files changed, 80 deletions(-) delete mode 100755 utils/travis/normal.sh delete mode 100755 utils/travis/pbuilder.sh diff --git a/utils/travis/normal.sh b/utils/travis/normal.sh deleted file mode 100755 index 7f45641d45f..00000000000 --- a/utils/travis/normal.sh +++ /dev/null @@ -1,45 +0,0 @@ -#!/bin/bash - -# Manual run: -# env CXX=g++-7 CC=gcc-7 utils/travis/normal.sh -# env CXX=clang++-5.0 CC=clang-5.0 utils/travis/normal.sh - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -CXX=${CXX=g++} -CC=${CC=gcc} - -set -e -set -x - -date - -# clean not used ~600mb -[ -n "$TRAVIS" ] && rm -rf .git contrib/poco/openssl - -ccache -s -ccache -M ${CCACHE_SIZE:=4G} -df -h - -date - -mkdir -p build -cd build -cmake $CUR_DIR/../.. -DCMAKE_CXX_COMPILER=`which $DEB_CXX $CXX` -DCMAKE_C_COMPILER=`which $DEB_CC $CC` \ - `# Does not optimize to speedup build, skip debug info to use less disk` \ - -DCMAKE_C_FLAGS_ADD="-O0 -g0" -DCMAKE_CXX_FLAGS_ADD="-O0 -g0" \ - `# ignore ccache disabler on trusty` \ - -DCMAKE_C_COMPILER_LAUNCHER=`which ccache` -DCMAKE_CXX_COMPILER_LAUNCHER=`which ccache` \ - `# Use all possible contrib libs from system` \ - -DUNBUNDLED=1 \ - `# Disable all features` \ - -DENABLE_CAPNP=0 -DENABLE_RDKAFKA=0 -DENABLE_EMBEDDED_COMPILER=0 -DUSE_INTERNAL_LLVM_LIBRARY=0 -DENABLE_JEMALLOC=0 -DENABLE_UNWIND=0 -DENABLE_MYSQL=0 -DENABLE_POCO_ODBC=0 -DENABLE_ODBC=0 $CMAKE_FLAGS - -ninja clickhouse-bundle - -# Skip tests: -# 00281 requires internal compiler -# 00428 requires sudo (not all vms allow this) -# 00385 runs infinitly (TODO: fix it) -[ ! ${TEST_RUN=1} ] || ( ( cd $CUR_DIR/../.. && env TEST_OPT="--skip long compile 00428 00385 $TEST_OPT" TEST_PORT_RANDOM= TEST_PERF= TEST_SERVER_STARTUP_WAIT=10 bash -x dbms/tests/clickhouse-test-server ) || ${TEST_TRUE=false} ) - -date diff --git a/utils/travis/pbuilder.sh b/utils/travis/pbuilder.sh deleted file mode 100755 index d993e8715b8..00000000000 --- a/utils/travis/pbuilder.sh +++ /dev/null @@ -1,35 +0,0 @@ -#!/bin/bash - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) - -# env CXX=clang++-5.0 CC=clang-5.0 DH_VERBOSE=1 utils/travis/pbuilder.sh - -set -e -set -x - -df -h - -date - -env TEST_RUN=${TEST_RUN=1} \ - TEST_PORT_RANDOM= \ - `# Skip tests:` \ - `# 00416 requires patched poco from contrib/` \ - TEST_OPT="--skip long pocopatch $TEST_OPT" \ - TEST_SSL="" `# Application: SSL context exception: Error loading certificate from file /etc/clickhouse-server/server.crt: No error -- when using system poco on artful` \ - TEST_TRUE=${TEST_TRUE=false} \ - `# travisci will not upload ccache cache after timeout (48min), use our less timeout` \ - PBUILDER_OPT="--timeout ${PBUILDER_TIMEOUT:=35m} $PBUILDER_OPT" \ - `# clang is faster than gcc` \ - DEB_CC=${DEB_CC=$CC} DEB_CXX=${DEB_CXX=$CXX} \ - CCACHE_SIZE=${CCACHE_SIZE:=4G} \ - `# Disable all features` \ - CMAKE_FLAGS="-DCMAKE_BUILD_TYPE=Debug -DUNBUNDLED=1 -DENABLE_CAPNP=0 -DENABLE_RDKAFKA=0 -DENABLE_JEMALLOC=0 -DENABLE_UNWIND=0 -DENABLE_MYSQL=0 -DENABLE_POCO_ODBC=0 -DENABLE_ODBC=0 -DUSE_INTERNAL_LLVM_LIBRARY=0 -DCMAKE_C_FLAGS_ADD='-O0 -g0' -DCMAKE_CXX_FLAGS_ADD='-O0 -g0' $CMAKE_FLAGS" \ - `# Use all possible contrib libs from system` \ - `# psmisc - killall` \ - EXTRAPACKAGES="psmisc clang-5.0 lld-5.0 liblld-5.0-dev libclang-5.0-dev liblld-5.0 libc++abi-dev libc++-dev libboost-program-options-dev libboost-system-dev libboost-filesystem-dev libboost-thread-dev zlib1g-dev liblz4-dev libdouble-conversion-dev libsparsehash-dev librdkafka-dev libpoco-dev libsparsehash-dev libgoogle-perftools-dev libzstd-dev libre2-dev libjemalloc-dev $EXTRAPACKAGES" \ - `# Travis trusty cant unpack bionic: E: debootstrap failed, TODO: check again, can be fixed` \ - DIST=${DIST=artful} \ - $CUR_DIR/../../release $RELEASE_OPT - -date From ba312c138bd16588ab19c396a66f8b1c4f8ee043 Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 23 Apr 2019 02:40:40 +0300 Subject: [PATCH 27/37] Parallel tests runner (#5076) * Parallel tests runner * Slightly better style * Fixes --- dbms/tests/clickhouse-test | 465 ++++++++++++++++++++----------------- 1 file changed, 248 insertions(+), 217 deletions(-) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 6b11331f451..f1819d12d9a 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -20,6 +20,8 @@ from errno import ESRCH import termcolor from random import random import commands +from multiprocessing import Pool +from contextlib import closing MESSAGES_TO_RETRY = [ @@ -87,23 +89,227 @@ def get_server_pid(server_tcp_port): except Exception as ex: return None +def dump_report(destination, suite, test_case, report): + if destination is not None: + destination_file = os.path.join(destination, suite, test_case + ".xml") + destination_dir = os.path.dirname(destination_file) + if not os.path.exists(destination_dir): + os.makedirs(destination_dir) + with open(destination_file, 'w') as report_file: + report_root = et.Element("testsuites", attrib = {'name': 'ClickHouse Tests'}) + report_suite = et.Element("testsuite", attrib = {"name": suite}) + report_suite.append(report) + report_root.append(report_suite) + report_file.write(et.tostring(report_root, encoding = "UTF-8", xml_declaration=True, pretty_print=True)) + +def colored(text, args, color=None, on_color=None, attrs=None): + if sys.stdout.isatty() or args.force_color: + return termcolor.colored(text, color, on_color, attrs) + else: + return text + + +SERVER_DIED = False +exit_code = 0 + + +#def run_tests_array(all_tests, suite, suite_dir, suite_tmp_dir, run_total): +def run_tests_array(all_tests_with_params): + all_tests, suite, suite_dir, suite_tmp_dir, run_total = all_tests_with_params + global SERVER_DIED + + OP_SQUARE_BRACKET = colored("[", args, attrs=['bold']) + CL_SQUARE_BRACKET = colored("]", args, attrs=['bold']) + + MSG_FAIL = OP_SQUARE_BRACKET + colored(" FAIL ", args, "red", attrs=['bold']) + CL_SQUARE_BRACKET + MSG_UNKNOWN = OP_SQUARE_BRACKET + colored(" UNKNOWN ", args, "yellow", attrs=['bold']) + CL_SQUARE_BRACKET + MSG_OK = OP_SQUARE_BRACKET + colored(" OK ", args, "green", attrs=['bold']) + CL_SQUARE_BRACKET + MSG_SKIPPED = OP_SQUARE_BRACKET + colored(" SKIPPED ", args, "cyan", attrs=['bold']) + CL_SQUARE_BRACKET + + passed_total = 0 + skipped_total = 0 + failures_total = 0 + failures = 0 + failures_chain = 0 + + if len(all_tests): + print("\nRunning {} {} tests.".format(len(all_tests), suite) + "\n") + + for case in all_tests: + if SERVER_DIED: + break + + case_file = os.path.join(suite_dir, case) + (name, ext) = os.path.splitext(case) + + report_testcase = et.Element("testcase", attrib = {"name": name}) + + try: + sys.stdout.write("{0:72}".format(name + ": ")) + if run_total == 1: + sys.stdout.flush() + + if args.skip and any(s in name for s in args.skip): + report_testcase.append(et.Element("skipped", attrib = {"message": "skip"})) + print(MSG_SKIPPED + " - skip") + skipped_total += 1 + elif not args.zookeeper and 'zookeeper' in name: + report_testcase.append(et.Element("skipped", attrib = {"message": "no zookeeper"})) + print(MSG_SKIPPED + " - no zookeeper") + skipped_total += 1 + elif not args.shard and 'shard' in name: + report_testcase.append(et.Element("skipped", attrib = {"message": "no shard"})) + print(MSG_SKIPPED + " - no shard") + skipped_total += 1 + elif not args.no_long and 'long' in name: + report_testcase.append(et.Element("skipped", attrib = {"message": "no long"})) + print(MSG_SKIPPED + " - no long") + skipped_total += 1 + else: + disabled_file = os.path.join(suite_dir, name) + '.disabled' + + if os.path.exists(disabled_file) and not args.disabled: + message = open(disabled_file, 'r').read() + report_testcase.append(et.Element("skipped", attrib = {"message": message})) + print(MSG_SKIPPED + " - " + message) + else: + + if args.testname: + clickhouse_proc = Popen(shlex.split(args.client_with_database), stdin=PIPE, stdout=PIPE, stderr=PIPE) + clickhouse_proc.communicate("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite)) + + reference_file = os.path.join(suite_dir, name) + '.reference' + stdout_file = os.path.join(suite_tmp_dir, name) + '.stdout' + stderr_file = os.path.join(suite_tmp_dir, name) + '.stderr' + + proc, stdout, stderr = run_single_test(args, ext, server_logs_level, case_file, stdout_file, stderr_file) + if proc.returncode is None: + try: + proc.kill() + except OSError as e: + if e.errno != ESRCH: + raise + + failure = et.Element("failure", attrib = {"message": "Timeout"}) + report_testcase.append(failure) + + failures += 1 + print("{0} - Timeout!".format(MSG_FAIL)) + else: + counter = 1 + while proc.returncode != 0 and need_retry(stderr): + proc, stdout, stderr = run_single_test(args, ext, server_logs_level, case_file, stdout_file, stderr_file) + sleep(2**counter) + counter += 1 + if counter > 6: + break + + if proc.returncode != 0: + failure = et.Element("failure", attrib = {"message": "return code {}".format(proc.returncode)}) + report_testcase.append(failure) + + stdout_element = et.Element("system-out") + stdout_element.text = et.CDATA(stdout) + report_testcase.append(stdout_element) + + failures += 1 + failures_chain += 1 + print("{0} - return code {1}".format(MSG_FAIL, proc.returncode)) + + if stderr: + stderr_element = et.Element("system-err") + stderr_element.text = et.CDATA(stderr) + report_testcase.append(stderr_element) + print(stderr.encode('utf-8')) + + if args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr: + SERVER_DIED = True + + elif stderr: + failure = et.Element("failure", attrib = {"message": "having stderror"}) + report_testcase.append(failure) + + stderr_element = et.Element("system-err") + stderr_element.text = et.CDATA(stderr) + report_testcase.append(stderr_element) + + failures += 1 + failures_chain += 1 + print("{0} - having stderror:\n{1}".format(MSG_FAIL, stderr.encode('utf-8'))) + elif 'Exception' in stdout: + failure = et.Element("error", attrib = {"message": "having exception"}) + report_testcase.append(failure) + + stdout_element = et.Element("system-out") + stdout_element.text = et.CDATA(stdout) + report_testcase.append(stdout_element) + + failures += 1 + failures_chain += 1 + print("{0} - having exception:\n{1}".format(MSG_FAIL, stdout.encode('utf-8'))) + elif not os.path.isfile(reference_file): + skipped = et.Element("skipped", attrib = {"message": "no reference file"}) + report_testcase.append(skipped) + print("{0} - no reference file".format(MSG_UNKNOWN)) + else: + result_is_different = subprocess.call(['diff', '-q', reference_file, stdout_file], stdout = PIPE) + + if result_is_different: + diff = Popen(['diff', '--unified', reference_file, stdout_file], stdout = PIPE).communicate()[0] + diff = unicode(diff, errors='replace', encoding='utf-8') + cat = Popen(['cat', '-vet'], stdin=PIPE, stdout=PIPE).communicate(input=diff.encode(encoding='utf-8', errors='replace'))[0] + + failure = et.Element("failure", attrib = {"message": "result differs with reference"}) + report_testcase.append(failure) + + stdout_element = et.Element("system-out") + try: + stdout_element.text = et.CDATA(diff) + except: + stdout_element.text = et.CDATA(remove_control_characters(diff)) + + report_testcase.append(stdout_element) + failures += 1 + print("{0} - result differs with reference:\n{1}".format(MSG_FAIL, cat.encode('utf-8'))) + else: + passed_total += 1 + failures_chain = 0 + print(MSG_OK) + if os.path.exists(stdout_file): + os.remove(stdout_file) + if os.path.exists(stderr_file): + os.remove(stderr_file) + except KeyboardInterrupt as e: + print(colored("Break tests execution", args, "red")) + raise e + except: + import traceback + exc_type, exc_value, tb = sys.exc_info() + error = et.Element("error", attrib = {"type": exc_type.__name__, "message": str(exc_value)}) + report_testcase.append(error) + + failures += 1 + print("{0} - Test internal error: {1}\n{2}\n{3}".format(MSG_FAIL, exc_type.__name__, exc_value, "\n".join(traceback.format_tb(tb, 10)))) + finally: + dump_report(args.output, suite, name, report_testcase) + + if failures_chain >= 20: + break + + failures_total = failures_total + failures + + if failures_total > 0: + print(colored("\nHaving {failures_total} errors! {passed_total} tests passed. {skipped_total} tests skipped.".format(passed_total = passed_total, skipped_total = skipped_total, failures_total = failures_total), args, "red", attrs=["bold"])) + exit_code = 1 + else: + print(colored("\n{passed_total} tests passed. {skipped_total} tests skipped.".format(passed_total = passed_total, skipped_total = skipped_total), args, "green", attrs=["bold"])) + +server_logs_level = "warning" + def main(args): - - SERVER_DIED = False - - def colored(text, color=None, on_color=None, attrs=None): - if sys.stdout.isatty() or args.force_color: - return termcolor.colored(text, color, on_color, attrs) - else: - return text - - OP_SQUARE_BRACKET = colored("[", attrs=['bold']) - CL_SQUARE_BRACKET = colored("]", attrs=['bold']) - - MSG_FAIL = OP_SQUARE_BRACKET + colored(" FAIL ", "red", attrs=['bold']) + CL_SQUARE_BRACKET - MSG_UNKNOWN = OP_SQUARE_BRACKET + colored(" UNKNOWN ", "yellow", attrs=['bold']) + CL_SQUARE_BRACKET - MSG_OK = OP_SQUARE_BRACKET + colored(" OK ", "green", attrs=['bold']) + CL_SQUARE_BRACKET - MSG_SKIPPED = OP_SQUARE_BRACKET + colored(" SKIPPED ", "cyan", attrs=['bold']) + CL_SQUARE_BRACKET + global SERVER_DIED + global exit_code + global server_logs_level def is_data_present(): clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) @@ -113,20 +319,6 @@ def main(args): return stdout.startswith('1') - - def dump_report(destination, suite, test_case, report): - if destination is not None: - destination_file = os.path.join(destination, suite, test_case + ".xml") - destination_dir = os.path.dirname(destination_file) - if not os.path.exists(destination_dir): - os.makedirs(destination_dir) - with open(destination_file, 'w') as report_file: - report_root = et.Element("testsuites", attrib = {'name': 'ClickHouse Tests'}) - report_suite = et.Element("testsuite", attrib = {"name": suite}) - report_suite.append(report) - report_root.append(report_suite) - report_file.write(et.tostring(report_root, encoding = "UTF-8", xml_declaration=True, pretty_print=True)) - base_dir = os.path.abspath(args.queries) tmp_dir = os.path.abspath(args.tmp) @@ -141,7 +333,6 @@ def main(args): # Force to print server warnings in stderr # Shell scripts could change logging level - server_logs_level = "warning" os.environ.setdefault("CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL", server_logs_level) if args.zookeeper is None: @@ -161,10 +352,6 @@ def main(args): else: args.shard = False - passed_total = 0 - skipped_total = 0 - failures_total = 0 - clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS " + args.database) if args.database != "test": @@ -206,8 +393,6 @@ def main(args): suite = suite_re_obj.group(1) if os.path.isdir(suite_dir): - failures = 0 - failures_chain = 0 if 'stateful' in suite and not is_data_present(): print("Won't run stateful tests because test data wasn't loaded.") continue @@ -236,203 +421,48 @@ def main(args): except ValueError: return 99997 - run_n, run_total = args.parallel.split('/') - run_n = float(run_n) - run_total = float(run_total) all_tests = os.listdir(suite_dir) all_tests = filter(lambda case: is_test_from_dir(suite_dir, case), all_tests) all_tests = sorted(filter(lambda case: re.search(args.test, case) if args.test else True, all_tests), key=key_func) + + run_n, run_total = args.parallel.split('/') + run_n = float(run_n) + run_total = float(run_total) tests_n = len(all_tests) - start = int(tests_n / run_total * (run_n - 1)) - if start > 0: - start = start + 1 - end = int(tests_n / run_total * (run_n)) - all_tests = all_tests[start : end] + if run_total > tests_n: + run_total = tests_n + if run_n > run_total: + continue - print("\nRunning {} {} tests.".format(tests_n, suite) + (" {} .. {} ".format(start, end) if run_total > 1 else "") + "\n") + jobs = args.jobs + if jobs > run_total: + run_total = jobs - for case in all_tests: - if SERVER_DIED: - break + all_tests_array = [] + for n in range(1, 1 + int(run_total)): + start = int(tests_n / run_total * (n - 1)) + end = int(tests_n / run_total * n) + all_tests_array.append([all_tests[start : end], suite, suite_dir, suite_tmp_dir, run_total]) - case_file = os.path.join(suite_dir, case) - (name, ext) = os.path.splitext(case) - - report_testcase = et.Element("testcase", attrib = {"name": name}) - - try: - sys.stdout.write("{0:72}".format(name + ": ")) - if run_total == 1: - sys.stdout.flush() - - if args.skip and any(s in name for s in args.skip): - report_testcase.append(et.Element("skipped", attrib = {"message": "skip"})) - print(MSG_SKIPPED + " - skip") - skipped_total += 1 - elif not args.zookeeper and 'zookeeper' in name: - report_testcase.append(et.Element("skipped", attrib = {"message": "no zookeeper"})) - print(MSG_SKIPPED + " - no zookeeper") - skipped_total += 1 - elif not args.shard and 'shard' in name: - report_testcase.append(et.Element("skipped", attrib = {"message": "no shard"})) - print(MSG_SKIPPED + " - no shard") - skipped_total += 1 - elif not args.no_long and 'long' in name: - report_testcase.append(et.Element("skipped", attrib = {"message": "no long"})) - print(MSG_SKIPPED + " - no long") - skipped_total += 1 - else: - disabled_file = os.path.join(suite_dir, name) + '.disabled' - - if os.path.exists(disabled_file) and not args.disabled: - message = open(disabled_file, 'r').read() - report_testcase.append(et.Element("skipped", attrib = {"message": message})) - print(MSG_SKIPPED + " - " + message) - else: - - if args.testname: - clickhouse_proc = Popen(shlex.split(args.client_with_database), stdin=PIPE, stdout=PIPE, stderr=PIPE) - clickhouse_proc.communicate("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite)) - - reference_file = os.path.join(suite_dir, name) + '.reference' - stdout_file = os.path.join(suite_tmp_dir, name) + '.stdout' - stderr_file = os.path.join(suite_tmp_dir, name) + '.stderr' - - proc, stdout, stderr = run_single_test(args, ext, server_logs_level, case_file, stdout_file, stderr_file) - if proc.returncode is None: - try: - proc.kill() - except OSError as e: - if e.errno != ESRCH: - raise - - failure = et.Element("failure", attrib = {"message": "Timeout"}) - report_testcase.append(failure) - - failures += 1 - print("{0} - Timeout!".format(MSG_FAIL)) - else: - counter = 1 - while proc.returncode != 0 and need_retry(stderr): - proc, stdout, stderr = run_single_test(args, ext, server_logs_level, case_file, stdout_file, stderr_file) - sleep(2**counter) - counter += 1 - if counter > 6: - break - - if proc.returncode != 0: - failure = et.Element("failure", attrib = {"message": "return code {}".format(proc.returncode)}) - report_testcase.append(failure) - - stdout_element = et.Element("system-out") - stdout_element.text = et.CDATA(stdout) - report_testcase.append(stdout_element) - - failures += 1 - failures_chain += 1 - print("{0} - return code {1}".format(MSG_FAIL, proc.returncode)) - - if stderr: - stderr_element = et.Element("system-err") - stderr_element.text = et.CDATA(stderr) - report_testcase.append(stderr_element) - print(stderr.encode('utf-8')) - - if args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr: - SERVER_DIED = True - - elif stderr: - failure = et.Element("failure", attrib = {"message": "having stderror"}) - report_testcase.append(failure) - - stderr_element = et.Element("system-err") - stderr_element.text = et.CDATA(stderr) - report_testcase.append(stderr_element) - - failures += 1 - failures_chain += 1 - print("{0} - having stderror:\n{1}".format(MSG_FAIL, stderr.encode('utf-8'))) - elif 'Exception' in stdout: - failure = et.Element("error", attrib = {"message": "having exception"}) - report_testcase.append(failure) - - stdout_element = et.Element("system-out") - stdout_element.text = et.CDATA(stdout) - report_testcase.append(stdout_element) - - failures += 1 - failures_chain += 1 - print("{0} - having exception:\n{1}".format(MSG_FAIL, stdout.encode('utf-8'))) - elif not os.path.isfile(reference_file): - skipped = et.Element("skipped", attrib = {"message": "no reference file"}) - report_testcase.append(skipped) - print("{0} - no reference file".format(MSG_UNKNOWN)) - else: - result_is_different = subprocess.call(['diff', '-q', reference_file, stdout_file], stdout = PIPE) - - if result_is_different: - diff = Popen(['diff', '--unified', reference_file, stdout_file], stdout = PIPE).communicate()[0] - diff = unicode(diff, errors='replace', encoding='utf-8') - cat = Popen(['cat', '-vet'], stdin=PIPE, stdout=PIPE).communicate(input=diff.encode(encoding='utf-8', errors='replace'))[0] - - failure = et.Element("failure", attrib = {"message": "result differs with reference"}) - report_testcase.append(failure) - - stdout_element = et.Element("system-out") - try: - stdout_element.text = et.CDATA(diff) - except: - stdout_element.text = et.CDATA(remove_control_characters(diff)) - - report_testcase.append(stdout_element) - failures += 1 - print("{0} - result differs with reference:\n{1}".format(MSG_FAIL, cat.encode('utf-8'))) - else: - passed_total += 1 - failures_chain = 0 - print(MSG_OK) - if os.path.exists(stdout_file): - os.remove(stdout_file) - if os.path.exists(stderr_file): - os.remove(stderr_file) - except KeyboardInterrupt as e: - print(colored("Break tests execution", "red")) - raise e - except: - import traceback - exc_type, exc_value, tb = sys.exc_info() - error = et.Element("error", attrib = {"type": exc_type.__name__, "message": str(exc_value)}) - report_testcase.append(error) - - failures += 1 - print("{0} - Test internal error: {1}\n{2}\n{3}".format(MSG_FAIL, exc_type.__name__, exc_value, "\n".join(traceback.format_tb(tb, 10)))) - finally: - dump_report(args.output, suite, name, report_testcase) - - if failures_chain >= 20: - break - - failures_total = failures_total + failures - - exit_code = 0 - if failures_total > 0: - print(colored("\nHaving {failures_total} errors! {passed_total} tests passed. {skipped_total} tests skipped.".format(passed_total = passed_total, skipped_total = skipped_total, failures_total = failures_total), "red", attrs=["bold"])) - exit_code = 1 - else: - print(colored("\n{passed_total} tests passed. {skipped_total} tests skipped.".format(passed_total = passed_total, skipped_total = skipped_total), "green", attrs=["bold"])) + if jobs > 1: + with closing(Pool(processes=jobs)) as pool: + pool.map(run_tests_array, all_tests_array) + pool.terminate() + else: + run_tests_array(all_tests_array[int(run_n)-1]) if args.hung_check: processlist = get_processlist(args.client_with_database) if processlist: server_pid = get_server_pid(os.getenv("CLICKHOUSE_PORT_TCP", '9000')) - print(colored("\nFound hung queries in processlist:", "red", attrs=["bold"])) + print(colored("\nFound hung queries in processlist:", args, "red", attrs=["bold"])) print(processlist) if server_pid: print("\nStacktraces of all threads:") print(get_stacktraces(server_pid)) exit_code = 1 else: - print(colored("\nNo queries hung.", "green", attrs=["bold"])) + print(colored("\nNo queries hung.", args, "green", attrs=["bold"])) sys.exit(exit_code) @@ -467,7 +497,8 @@ if __name__ == '__main__': parser.add_argument('--hung-check', action='store_true', default=False) parser.add_argument('--force-color', action='store_true', default=False) parser.add_argument('--database', default='test', help='Default database for tests') - parser.add_argument('--parallel', default='1/1', help='Parralel test run number/total') + parser.add_argument('--parallel', default='1/1', help='One parallel test run number/total') + parser.add_argument('-j', '--jobs', default=1, help='Run all tests in parallel', type=int) parser.add_argument('--no-stateless', action='store_true', help='Disable all stateless tests') parser.add_argument('--no-stateful', action='store_true', help='Disable all stateful tests') From 0060650bb0b3354eb267a053c8277e6beb286269 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 23 Apr 2019 13:41:01 +0300 Subject: [PATCH 28/37] Fix stress test exit code check and add brotli to image --- docker/test/stress/Dockerfile | 4 +++- docker/test/stress/stress | 4 ++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/docker/test/stress/Dockerfile b/docker/test/stress/Dockerfile index b2504988249..aea8509cfe3 100644 --- a/docker/test/stress/Dockerfile +++ b/docker/test/stress/Dockerfile @@ -18,7 +18,9 @@ RUN apt-get update -y \ openssl \ netcat-openbsd \ telnet \ - llvm-8 + llvm-8 \ + brotli + COPY ./stress /stress COPY log_queries.xml /etc/clickhouse-server/users.d/log_queries.xml diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 9a8c236234c..8fb351c1395 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -11,7 +11,7 @@ import time def run_perf_test(cmd, xmls_path, output_folder): output_path = os.path.join(output_folder, "perf_stress_run.txt") f = open(output_path, 'w') - p = Popen("{} --skip-tags=long --r {}".format(cmd, xmls_path), shell=True, stdout=f, stderr=f) + p = Popen("{} --skip-tags=long --recursive --input-files {}".format(cmd, xmls_path), shell=True, stdout=f, stderr=f) return p def run_func_test(cmd, output_prefix, num_processes): @@ -55,7 +55,7 @@ if __name__ == "__main__": while True: retcodes = [] for p in func_pipes: - if p.poll(): + if p.poll() is not None: retcodes.append(p.returncode) if len(retcodes) == len(func_pipes): break From 8d808aedf3ab175ad621a0c938684023b29b6648 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 23 Apr 2019 14:50:43 +0300 Subject: [PATCH 29/37] Fix bug in s3uploader --- utils/s3tools/s3uploader | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/s3tools/s3uploader b/utils/s3tools/s3uploader index db3f7cb2335..25d4abbd375 100755 --- a/utils/s3tools/s3uploader +++ b/utils/s3tools/s3uploader @@ -124,7 +124,7 @@ if __name__ == "__main__": args.dataset_name, 'partitions', os.path.basename(file_path)) elif args.s3_path is not None: s3_path = os.path.join( - args.dataset_name, s3_path, os.path.base_name(file_path)) + args.dataset_name, args.s3_path, os.path.basename(file_path)) else: raise Exception("Don't know s3-path to upload") From 29c34333224f8ae840d8b63bc137d67b71a0bc90 Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 23 Apr 2019 15:51:27 +0300 Subject: [PATCH 30/37] Freebsd build fixes (#5083) --- cmake/find_execinfo.cmake | 7 +++--- dbms/CMakeLists.txt | 3 +-- .../clang/Compiler-5.0.0/CMakeLists.txt | 2 +- .../clang/Compiler-6.0.0/CMakeLists.txt | 2 +- .../clang/Compiler-7.0.0/CMakeLists.txt | 2 +- .../Compiler-7.0.0bundled/CMakeLists.txt | 2 +- dbms/tests/clickhouse-test | 3 ++- libs/libcommon/CMakeLists.txt | 6 ++++- libs/libdaemon/CMakeLists.txt | 2 +- libs/libmysqlxx/cmake/find_mysqlclient.cmake | 22 +++++++++---------- 10 files changed, 27 insertions(+), 24 deletions(-) diff --git a/cmake/find_execinfo.cmake b/cmake/find_execinfo.cmake index 650d279983c..85cc5cf951a 100644 --- a/cmake/find_execinfo.cmake +++ b/cmake/find_execinfo.cmake @@ -1,9 +1,8 @@ if (OS_FREEBSD) find_library (EXECINFO_LIBRARY execinfo) find_library (ELF_LIBRARY elf) - message (STATUS "Using execinfo: ${EXECINFO_LIBRARY}") - message (STATUS "Using elf: ${ELF_LIBRARY}") + set (EXECINFO_LIBRARIES ${EXECINFO_LIBRARY} ${ELF_LIBRARY}) + message (STATUS "Using execinfo: ${EXECINFO_LIBRARIES}") else () - set (EXECINFO_LIBRARY "") - set (ELF_LIBRARY "") + set (EXECINFO_LIBRARIES "") endif () diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 2c9bfa48605..63e97f4e061 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -194,8 +194,7 @@ target_link_libraries (clickhouse_common_io ${CITYHASH_LIBRARIES} PRIVATE ${ZLIB_LIBRARIES} - ${EXECINFO_LIBRARY} - ${ELF_LIBRARY} + ${EXECINFO_LIBRARIES} PUBLIC ${Boost_SYSTEM_LIBRARY} PRIVATE diff --git a/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt b/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt index 7fe0cd92ef7..e0171630bf2 100644 --- a/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt +++ b/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt @@ -46,7 +46,7 @@ LLVMSupport #PollyISL #PollyPPCG -PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARY} Threads::Threads +PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARIES} Threads::Threads ${MALLOC_LIBRARIES} ${GLIBC_COMPATIBILITY_LIBRARIES} ${MEMCPY_LIBRARIES} diff --git a/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt b/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt index b96bdb0647a..bac622ab09e 100644 --- a/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt +++ b/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt @@ -46,7 +46,7 @@ ${REQUIRED_LLVM_LIBRARIES} #PollyISL #PollyPPCG -PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARY} Threads::Threads +PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARIES} Threads::Threads ${MALLOC_LIBRARIES} ${GLIBC_COMPATIBILITY_LIBRARIES} ${MEMCPY_LIBRARIES} diff --git a/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt b/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt index 8b6ba6be994..35e23cc6b46 100644 --- a/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt +++ b/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt @@ -42,7 +42,7 @@ lldCore ${REQUIRED_LLVM_LIBRARIES} -PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARY} Threads::Threads +PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARIES} Threads::Threads ${MALLOC_LIBRARIES} ${GLIBC_COMPATIBILITY_LIBRARIES} ${MEMCPY_LIBRARIES} diff --git a/dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt b/dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt index d0ccc8d672c..d03052ffc28 100644 --- a/dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt +++ b/dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt @@ -42,7 +42,7 @@ lldCore ${REQUIRED_LLVM_LIBRARIES} -PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARY} Threads::Threads +PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARIES} Threads::Threads ${MALLOC_LIBRARIES} ${GLIBC_COMPATIBILITY_LIBRARIES} ${MEMCPY_LIBRARIES} diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index f1819d12d9a..7612c415373 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -116,6 +116,7 @@ exit_code = 0 #def run_tests_array(all_tests, suite, suite_dir, suite_tmp_dir, run_total): def run_tests_array(all_tests_with_params): all_tests, suite, suite_dir, suite_tmp_dir, run_total = all_tests_with_params + global exit_code global SERVER_DIED OP_SQUARE_BRACKET = colored("[", args, attrs=['bold']) @@ -393,7 +394,7 @@ def main(args): suite = suite_re_obj.group(1) if os.path.isdir(suite_dir): - if 'stateful' in suite and not is_data_present(): + if 'stateful' in suite and not args.no_stateful and not is_data_present(): print("Won't run stateful tests because test data wasn't loaded.") continue if 'stateless' in suite and args.no_stateless: diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index 999290996a9..02199c21a4f 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -91,9 +91,13 @@ target_include_directories (common BEFORE PRIVATE ${CCTZ_INCLUDE_DIR}) target_include_directories (common PUBLIC ${COMMON_INCLUDE_DIR}) if (NOT USE_INTERNAL_BOOST_LIBRARY) - target_include_directories (common BEFORE PUBLIC ${Boost_INCLUDE_DIRS}) + target_include_directories (common SYSTEM BEFORE PUBLIC ${Boost_INCLUDE_DIRS}) endif () +if(NOT USE_INTERNAL_POCO_LIBRARY) + target_include_directories (common SYSTEM BEFORE PUBLIC ${Poco_Foundation_INCLUDE_DIR}) +endif() + target_link_libraries (common PUBLIC ${Poco_Foundation_LIBRARY} diff --git a/libs/libdaemon/CMakeLists.txt b/libs/libdaemon/CMakeLists.txt index eb73f1cda33..181030009b0 100644 --- a/libs/libdaemon/CMakeLists.txt +++ b/libs/libdaemon/CMakeLists.txt @@ -22,4 +22,4 @@ endif () target_include_directories (daemon PUBLIC include) -target_link_libraries (daemon PRIVATE clickhouse_common_io clickhouse_common_config common ${Poco_Net_LIBRARY} ${Poco_Util_LIBRARY} ${EXECINFO_LIBRARY} ${ELF_LIBRARY}) +target_link_libraries (daemon PRIVATE clickhouse_common_io clickhouse_common_config common ${Poco_Net_LIBRARY} ${Poco_Util_LIBRARY} ${EXECINFO_LIBRARIES}) diff --git a/libs/libmysqlxx/cmake/find_mysqlclient.cmake b/libs/libmysqlxx/cmake/find_mysqlclient.cmake index 71cb2bfeb1d..98b42a0a9b4 100644 --- a/libs/libmysqlxx/cmake/find_mysqlclient.cmake +++ b/libs/libmysqlxx/cmake/find_mysqlclient.cmake @@ -1,17 +1,17 @@ -option (ENABLE_MYSQL "Enable MySQL" ${OS_LINUX}) -if (OS_LINUX) - option (USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient library instead of bundled" ${NOT_UNBUNDLED}) -else () - option (USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient library instead of bundled" OFF) -endif () +option(ENABLE_MYSQL "Enable MySQL" 1) +if(ENABLE_MYSQL) + if(OS_LINUX) + option(USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient library instead of bundled" ${NOT_UNBUNDLED}) + else() + option(USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient library instead of bundled" OFF) + endif() -if (USE_INTERNAL_MYSQL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mariadb-connector-c/README.md") - message (WARNING "submodule contrib/mariadb-connector-c is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_MYSQL_LIBRARY 0) -endif () + if(USE_INTERNAL_MYSQL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mariadb-connector-c/README.md") + message(WARNING "submodule contrib/mariadb-connector-c is missing. to fix try run: \n git submodule update --init --recursive") + set(USE_INTERNAL_MYSQL_LIBRARY 0) + endif() -if (ENABLE_MYSQL) if (USE_INTERNAL_MYSQL_LIBRARY) set (MYSQLCLIENT_LIBRARIES mysqlclient) set (USE_MYSQL 1) From fe7114282704951250ab033a8a18a548cff68709 Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 23 Apr 2019 17:02:26 +0300 Subject: [PATCH 31/37] Fix segfault in performance-test when no options specified --- dbms/programs/performance-test/PerformanceTestSuite.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/programs/performance-test/PerformanceTestSuite.cpp b/dbms/programs/performance-test/PerformanceTestSuite.cpp index 7a373d7bfba..9daf1b86ce7 100644 --- a/dbms/programs/performance-test/PerformanceTestSuite.cpp +++ b/dbms/programs/performance-test/PerformanceTestSuite.cpp @@ -298,6 +298,8 @@ std::unordered_map> getTestQueryIndexes(co { std::unordered_map> result; const auto & options = parsed_opts.options; + if (options.empty()) + return result; for (size_t i = 0; i < options.size() - 1; ++i) { const auto & opt = options[i]; From b52f1257a9e768925e808413eddd684b083d8bcb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 23 Apr 2019 19:48:46 +0300 Subject: [PATCH 32/37] Removed useless include --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 075a2e3c21f..06a8b93f8e2 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -36,7 +36,6 @@ #include #include -#include #include #include From 5aacb7505ded126d9aaa94d3a084a092766c2167 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E9=A3=8E=E5=95=B8?= Date: Tue, 23 Apr 2019 15:25:24 -0500 Subject: [PATCH 33/37] Zhdocs: SummingMergeTree and Dictionary (#5087) --- .../table_engines/summingmergetree.md | 2 +- .../zh/operations/table_engines/dictionary.md | 18 ++--- .../table_engines/summingmergetree.md | 78 +++++++++---------- 3 files changed, 49 insertions(+), 49 deletions(-) diff --git a/docs/en/operations/table_engines/summingmergetree.md b/docs/en/operations/table_engines/summingmergetree.md index a5458a5a1e0..807476861f2 100644 --- a/docs/en/operations/table_engines/summingmergetree.md +++ b/docs/en/operations/table_engines/summingmergetree.md @@ -86,7 +86,7 @@ SELECT key, sum(value) FROM summtt GROUP BY key ``` -## Data Processing +## Data Processing {#data-processing} When data are inserted into a table, they are saved as-is. Clickhouse merges the inserted parts of data periodically and this is when rows with the same primary key are summed and replaced with one for each resulting part of data. diff --git a/docs/zh/operations/table_engines/dictionary.md b/docs/zh/operations/table_engines/dictionary.md index fa2d6cbc7ce..951019161bc 100644 --- a/docs/zh/operations/table_engines/dictionary.md +++ b/docs/zh/operations/table_engines/dictionary.md @@ -1,9 +1,9 @@ # Dictionary -The `Dictionary` engine displays the dictionary data as a ClickHouse table. +`Dictionary` 引擎将字典数据展示为一个ClickHouse的表。 -As an example, consider a dictionary of `products` with the following configuration: +例如,考虑使用一个具有以下配置的 `products` 字典: ```xml @@ -36,7 +36,7 @@ As an example, consider a dictionary of `products` with the following configurat ``` -Query the dictionary data: +查询字典中的数据: ``` sql select name, type, key, attribute.names, attribute.types, bytes_allocated, element_count,source from system.dictionaries where name = 'products'; @@ -60,17 +60,17 @@ WHERE name = 'products' └──────────┴──────┴────────┴─────────────────┴─────────────────┴─────────────────┴───────────────┴─────────────────┘ ``` -You can use the [dictGet*](../../query_language/functions/ext_dict_functions.md#ext_dict_functions) function to get the dictionary data in this format. +你可以使用 [dictGet*](../../query_language/functions/ext_dict_functions.md#ext_dict_functions) 函数来获取这种格式的字典数据。 -This view isn't helpful when you need to get raw data, or when performing a `JOIN` operation. For these cases, you can use the `Dictionary` engine, which displays the dictionary data in a table. +当你需要获取原始数据,或者是想要使用 `JOIN` 操作的时候,这种视图并没有什么帮助。对于这些情况,你可以使用 `Dictionary` 引擎,它可以将字典数据展示在表中。 -Syntax: +语法: ``` CREATE TABLE %table_name% (%fields%) engine = Dictionary(%dictionary_name%)` ``` -Usage example: +示例: ``` sql create table products (product_id UInt64, title String) Engine = Dictionary(products); @@ -89,7 +89,7 @@ Ok. 0 rows in set. Elapsed: 0.004 sec. ``` -Take a look at what's in the table. +看一看表中的内容。 ``` sql select * from products limit 1; @@ -108,4 +108,4 @@ LIMIT 1 ``` -[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/dictionary/) +[来源文章](https://clickhouse.yandex/docs/en/operations/table_engines/dictionary/) diff --git a/docs/zh/operations/table_engines/summingmergetree.md b/docs/zh/operations/table_engines/summingmergetree.md index 82664cc2d9b..5f529718ae6 100644 --- a/docs/zh/operations/table_engines/summingmergetree.md +++ b/docs/zh/operations/table_engines/summingmergetree.md @@ -1,11 +1,11 @@ # SummingMergeTree -The engine inherits from [MergeTree](mergetree.md). The difference is that when merging data parts for `SummingMergeTree` tables ClickHouse replaces all the rows with the same primary key with one row which contains summarized values for the columns with the numeric data type. If the primary key is composed in a way that a single key value corresponds to large number of rows, this significantly reduces storage volume and speeds up data selection. +该引擎继承自 [MergeTree](mergetree.md)。区别在于,当合并 `SummingMergeTree` 表的数据片段时,ClickHouse 会把所有具有相同主键的行合并为一行,该行包含了被合并的行中具有数值数据类型的列的汇总值。如果主键的组合方式使得单个键值对应于大量的行,则可以显著的减少存储空间并加快数据查询的速度。 -We recommend to use the engine together with `MergeTree`. Store complete data in `MergeTree` table, and use `SummingMergeTree` for aggregated data storing, for example, when preparing reports. Such an approach will prevent you from losing valuable data due to an incorrectly composed primary key. +我们推荐将该引擎和 `MergeTree` 一起使用。例如,在准备做报告的时候,将完整的数据存储在 `MergeTree` 表中,并且使用 `SummingMergeTree` 来存储聚合数据。这种方法可以使你避免因为使用不正确的主键组合方式而丢失有价值的数据。 -## Creating a Table +## 建表 ``` CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] @@ -20,23 +20,23 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [SETTINGS name=value, ...] ``` -For a description of request parameters, see [request description](../../query_language/create.md). +请求参数的描述,参考 [请求描述](../../query_language/create.md)。 -**Parameters of SummingMergeTree** +**SummingMergeTree 的参数** -- `columns` - a tuple with the names of columns where values will be summarized. Optional parameter. -The columns must be of a numeric type and must not be in the primary key. +- `columns` - 包含了将要被汇总的列的列名的元组。可选参数。 +所选的列必须是数值类型,并且不可位于主键中。 - If `columns` not specified, ClickHouse summarizes the values in all columns with a numeric data type that are not in the primary key. + 如果没有指定 `columns`,ClickHouse 会把所有不在主键中的数值类型的列都进行汇总。 -**Query clauses** +**子句** -When creating a `SummingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table. +创建 `SummingMergeTree` 表时,需要与创建 `MergeTree` 表时相同的子句。 -
Deprecated Method for Creating a Table +
已弃用的建表方法 -!!! attention - Do not use this method in new projects and, if possible, switch the old projects to the method described above. +!!! 注意 + 不要在新项目中使用该方法,可能的话,请将旧项目切换到上述方法。 ``` CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] @@ -47,14 +47,14 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ) ENGINE [=] SummingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, [columns]) ``` -All of the parameters excepting `columns` have the same meaning as in `MergeTree`. +除 `columns` 外的所有参数都与 `MergeTree` 中的含义相同。 -- `columns` — tuple with names of columns values of which will be summarized. Optional parameter. For a description, see the text above. +- `columns` — 包含将要被汇总的列的列名的元组。可选参数。有关说明,请参阅上面的文字。
-## Usage Example +## 用法示例 -Consider the following table: +考虑如下的表: ```sql CREATE TABLE summtt @@ -66,13 +66,13 @@ ENGINE = SummingMergeTree() ORDER BY key ``` -Insert data to it: +向其中插入数据: ``` :) INSERT INTO summtt Values(1,1),(1,2),(2,1) ``` -ClickHouse may sum all the rows not completely ([see below](#data-processing)), so we use an aggregate function `sum` and `GROUP BY` clause in the query. +ClickHouse可能不会完整的汇总所有行([见下文](#data-processing)),因此我们在查询中使用了聚合函数 `sum` 和 `GROUP BY` 子句。 ```sql SELECT key, sum(value) FROM summtt GROUP BY key @@ -86,38 +86,38 @@ SELECT key, sum(value) FROM summtt GROUP BY key ``` -## Data Processing +## 数据处理 {#data-processing} -When data are inserted into a table, they are saved as-is. Clickhouse merges the inserted parts of data periodically and this is when rows with the same primary key are summed and replaced with one for each resulting part of data. +当数据被插入到表中时,他们将被原样保存。ClickHouse 定期合并插入的数据片段,并在这个时候对所有具有相同主键的行中的列进行汇总,将这些行替换为包含汇总数据的一行记录。 -ClickHouse can merge the data parts so that different resulting parts of data cat consist rows with the same primary key, i.e. the summation will be incomplete. Therefore (`SELECT`) an aggregate function [sum()](../../query_language/agg_functions/reference.md#agg_function-sum) and `GROUP BY` clause should be used in a query as described in the example above. +ClickHouse 会按片段合并数据,以至于不同的数据片段中会包含具有相同主键的行,即单个汇总片段将会是不完整的。因此,聚合函数 [sum()](../../query_language/agg_functions/reference.md#agg_function-sum) 和 `GROUP BY` 子句应该在(`SELECT`)查询语句中被使用,如上文中的例子所述。 -### Common rules for summation +### 汇总的通用规则 -The values in the columns with the numeric data type are summarized. The set of columns is defined by the parameter `columns`. +列中数值类型的值会被汇总。这些列的集合在参数 `columns` 中被定义。 -If the values were 0 in all of the columns for summation, the row is deleted. +如果用于汇总的所有列中的值均为0,则该行会被删除。 -If column is not in the primary key and is not summarized, an arbitrary value is selected from the existing ones. +如果列不在主键中且无法被汇总,则会在现有的值中任选一个。 -The values are not summarized for columns in the primary key. +主键所在的列中的值不会被汇总。 -### The Summation in the AggregateFunction Columns +### AggregateFunction 列中的汇总 -For columns of [AggregateFunction type](../../data_types/nested_data_structures/aggregatefunction.md) ClickHouse behaves as [AggregatingMergeTree](aggregatingmergetree.md) engine aggregating according to the function. +对于 [AggregateFunction 类型](../../data_types/nested_data_structures/aggregatefunction.md)的列,ClickHouse 根据对应函数表现为 [AggregatingMergeTree](aggregatingmergetree.md) 引擎的聚合。 -### Nested Structures +### 嵌套结构 -Table can have nested data structures that are processed in a special way. +表中可以具有以特殊方式处理的嵌套数据结构。 -If the name of a nested table ends with `Map` and it contains at least two columns that meet the following criteria: +如果嵌套表的名称以 `Map` 结尾,并且包含至少两个符合以下条件的列: -- the first column is numeric `(*Int*, Date, DateTime)`, let's call it `key`, -- the other columns are arithmetic `(*Int*, Float32/64)`, let's call it `(values...)`, +- 第一列是数值类型 `(*Int*, Date, DateTime)`,我们称之为 `key`, +- 其他的列是可计算的 `(*Int*, Float32/64)`,我们称之为 `(values...)`, -then this nested table is interpreted as a mapping of `key => (values...)`, and when merging its rows, the elements of two data sets are merged by `key` with a summation of the corresponding `(values...)`. +然后这个嵌套表会被解释为一个 `key => (values...)` 的映射,当合并它们的行时,两个数据集中的元素会被根据 `key` 合并为相应的 `(values...)` 的汇总值。 -Examples: +示例: ``` [(1, 100)] + [(2, 150)] -> [(1, 100), (2, 150)] @@ -126,8 +126,8 @@ Examples: [(1, 100), (2, 150)] + [(1, -100)] -> [(2, 150)] ``` -When requesting data, use the [sumMap(key, value)](../../query_language/agg_functions/reference.md) function for aggregation of `Map`. +请求数据时,使用 [sumMap(key, value)](../../query_language/agg_functions/reference.md) 函数来对 `Map` 进行聚合。 -For nested data structure, you do not need to specify its columns in the tuple of columns for summation. +对于嵌套数据结构,你无需在列的元组中指定列以进行汇总。 -[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/summingmergetree/) +[来源文章](https://clickhouse.yandex/docs/en/operations/table_engines/summingmergetree/) From 450e5974b73b0e3dbe60a471f325ef3906d25ee4 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 24 Apr 2019 09:42:42 +0300 Subject: [PATCH 34/37] Add link to GitHub issues to Contacts section --- website/index.html | 2 ++ 1 file changed, 2 insertions(+) diff --git a/website/index.html b/website/index.html index 90e25bb725d..134eb957d46 100644 --- a/website/index.html +++ b/website/index.html @@ -444,6 +444,8 @@ clickhouse-client rel="external nofollow" target="_blank">English or in Russian. +
  • Open GitHub issue if you have a bug report or feature request.
  • Follow official Twitter account.
  • From 312f8cb6376ed91fd369c09f553c2bf2063caf1a Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 24 Apr 2019 09:43:51 +0300 Subject: [PATCH 35/37] Swap list items at website front page --- website/index.html | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/website/index.html b/website/index.html index 134eb957d46..e6ebb8e6edd 100644 --- a/website/index.html +++ b/website/index.html @@ -444,11 +444,11 @@ clickhouse-client rel="external nofollow" target="_blank">English or in Russian. -
  • Open GitHub issue if you have a bug report or feature request.
  • Follow official Twitter account.
  • +
  • Open GitHub issue if you have a bug report or feature request.
  • Or email Yandex ClickHouse team directly at turn on JavaScript to see email address. You can also fill this form to meet us in person.
  • From 9440e1e61a850fa93a157baed0ecae74fc98eea1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E9=A3=8E=E5=95=B8?= Date: Wed, 24 Apr 2019 01:51:01 -0500 Subject: [PATCH 36/37] translate ReplacingMergeTree and optimize zh attention format (#5094) --- docs/zh/operations/table_engines/mergetree.md | 2 +- .../table_engines/replacingmergetree.md | 34 +++++++++---------- .../table_engines/summingmergetree.md | 6 ++-- 3 files changed, 21 insertions(+), 21 deletions(-) diff --git a/docs/zh/operations/table_engines/mergetree.md b/docs/zh/operations/table_engines/mergetree.md index c8febd0a21b..5ddf837708a 100644 --- a/docs/zh/operations/table_engines/mergetree.md +++ b/docs/zh/operations/table_engines/mergetree.md @@ -87,7 +87,7 @@ ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDa
    已弃用的建表方法 -!!! 注意 +!!! attention "注意" 不要在新版项目中使用该方法,可能的话,请将旧项目切换到上述方法。 ``` diff --git a/docs/zh/operations/table_engines/replacingmergetree.md b/docs/zh/operations/table_engines/replacingmergetree.md index 568f94822dd..a9bb9d65b6d 100644 --- a/docs/zh/operations/table_engines/replacingmergetree.md +++ b/docs/zh/operations/table_engines/replacingmergetree.md @@ -1,12 +1,12 @@ # ReplacingMergeTree -The engine differs from [MergeTree](mergetree.md) in that it removes duplicate entries with the same primary key value. +该引擎和[MergeTree](mergetree.md)的不同之处在于它会删除具有相同主键的重复项。 -Data deduplication occurs only during a merge. Merging occurs in the background at an unknown time, so you can't plan for it. Some of the data may remain unprocessed. Although you can run an unscheduled merge using the `OPTIMIZE` query, don't count on using it, because the `OPTIMIZE` query will read and write a large amount of data. +数据的去重只会在合并的过程中出现。合并会在未知的时间在后台进行,因此你无法预先作出计划。有一些数据可能仍未被处理。尽管你可以调用 `OPTIMIZE` 语句发起计划外的合并,但请不要指望使用它,因为 `OPTIMIZE` 语句会引发对大量数据的读和写。 -Thus, `ReplacingMergeTree` is suitable for clearing out duplicate data in the background in order to save space, but it doesn't guarantee the absence of duplicates. +因此,`ReplacingMergeTree` 适用于在后台清除重复的数据以节省空间,但是它不保证没有重复的数据出现。 -## Creating a Table +## 建表 ```sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] @@ -21,24 +21,24 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [SETTINGS name=value, ...] ``` -For a description of request parameters, see [request description](../../query_language/create.md). +请求参数的描述,参考[请求参数](../../query_language/create.md)。 **ReplacingMergeTree Parameters** -- `ver` — column with version. Type `UInt*`, `Date` or `DateTime`. Optional parameter. +- `ver` — 版本列。类型为 `UInt*`, `Date` 或 `DateTime`。可选参数。 - When merging, `ReplacingMergeTree` from all the rows with the same primary key leaves only one: - - Last in the selection, if `ver` not set. - - With the maximum version, if `ver` specified. + 合并的时候,`ReplacingMergeTree` 从所有具有相同主键的行中选择一行留下: + - 如果 `ver` 列未指定,选择最后一条。 + - 如果 `ver` 列已指定,选择 `ver` 值最大的版本。 -**Query clauses** +**子句** -When creating a `ReplacingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table. +创建 `ReplacingMergeTree` 表时,需要与创建 `MergeTree` 表时相同的[子句](mergetree.md)。 -
    Deprecated Method for Creating a Table +
    已弃用的建表方法 -!!! attention - Do not use this method in new projects and, if possible, switch the old projects to the method described above. +!!! attention "注意" + 不要在新项目中使用该方法,可能的话,请将旧项目切换到上述方法。 ```sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] @@ -49,10 +49,10 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ) ENGINE [=] ReplacingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, [ver]) ``` -All of the parameters excepting `ver` have the same meaning as in `MergeTree`. +除了 `ver` 的所有参数都与 `MergeTree` 中的含义相同。 -- `ver` - column with the version. Optional parameter. For a description, see the text above. +- `ver` - 版本列。可选参数,有关说明,请参阅上文。
    -[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/replacingmergetree/) +[来源文章](https://clickhouse.yandex/docs/en/operations/table_engines/replacingmergetree/) diff --git a/docs/zh/operations/table_engines/summingmergetree.md b/docs/zh/operations/table_engines/summingmergetree.md index 5f529718ae6..ab3a50765e2 100644 --- a/docs/zh/operations/table_engines/summingmergetree.md +++ b/docs/zh/operations/table_engines/summingmergetree.md @@ -31,11 +31,11 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] **子句** -创建 `SummingMergeTree` 表时,需要与创建 `MergeTree` 表时相同的子句。 +创建 `SummingMergeTree` 表时,需要与创建 `MergeTree` 表时相同的[子句](mergetree.md)。
    已弃用的建表方法 -!!! 注意 +!!! attention "注意" 不要在新项目中使用该方法,可能的话,请将旧项目切换到上述方法。 ``` @@ -49,7 +49,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] 除 `columns` 外的所有参数都与 `MergeTree` 中的含义相同。 -- `columns` — 包含将要被汇总的列的列名的元组。可选参数。有关说明,请参阅上面的文字。 +- `columns` — 包含将要被汇总的列的列名的元组。可选参数。有关说明,请参阅上文。
    ## 用法示例 From 566bf557b8c35049e8a85df1ddf1d05df39e3cd0 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 24 Apr 2019 11:08:21 +0300 Subject: [PATCH 37/37] fix mistype --- website/index.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/index.html b/website/index.html index e6ebb8e6edd..d0a95c73120 100644 --- a/website/index.html +++ b/website/index.html @@ -448,7 +448,7 @@ clickhouse-client href="https://twitter.com/ClickHouseDB" rel="external nofollow" target="_blank">Twitter account.
  • Open GitHub issue if you have a bug report or feature request.
  • + rel="external nofollow" target="_blank">GitHub issue if you have a bug report or feature request.
  • Or email Yandex ClickHouse team directly at turn on JavaScript to see email address. You can also fill this form to meet us in person.