From 5cbbcd9cdb726c49d85f37b56931a82f72a3c8c8 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 19 Nov 2019 09:44:10 +0300 Subject: [PATCH 01/42] Added disk info to `system.detached_parts`. --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 12 +++++++++++- dbms/src/Storages/MergeTree/MergeTreeData.h | 4 ++++ dbms/src/Storages/MergeTree/MergeTreePartInfo.h | 2 ++ .../Storages/System/StorageSystemDetachedParts.cpp | 2 ++ 4 files changed, 19 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 728f69f477e..802b2dce25c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -2933,7 +2933,7 @@ MergeTreeData::getDetachedParts() const { std::vector res; - for (const String & path : getDataPaths()) + for (const auto & [path, disk] : getDataPathsWithDisks()) { for (Poco::DirectoryIterator it(path + "detached"); it != Poco::DirectoryIterator(); ++it) @@ -2944,6 +2944,7 @@ MergeTreeData::getDetachedParts() const auto & part = res.back(); DetachedPartInfo::tryParseDetachedPartName(dir_name, part, format_version); + part.disk = disk->getName(); } } return res; @@ -3327,6 +3328,15 @@ Strings MergeTreeData::getDataPaths() const return res; } +MergeTreeData::PathsWithDisks MergeTreeData::getDataPathsWithDisks() const +{ + PathsWithDisks res; + auto disks = storage_policy->getDisks(); + for (const auto & disk : disks) + res.emplace_back(getFullPathOnDisk(disk), disk); + return res; +} + void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & with_name, const Context & context) { String clickhouse_path = Poco::Path(context.getPath()).makeAbsolute().toString(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index b03cbd8cb70..464cfcea52c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -669,6 +669,10 @@ public: Strings getDataPaths() const override; + using PathWithDisk = std::pair; + using PathsWithDisks = std::vector; + PathsWithDisks getDataPathsWithDisks() const; + /// Reserves space at least 1MB DiskSpace::ReservationPtr reserveSpace(UInt64 expected_size); diff --git a/dbms/src/Storages/MergeTree/MergeTreePartInfo.h b/dbms/src/Storages/MergeTree/MergeTreePartInfo.h index 9fe0fbab533..2cf423f325a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/dbms/src/Storages/MergeTree/MergeTreePartInfo.h @@ -95,6 +95,8 @@ struct DetachedPartInfo : public MergeTreePartInfo String dir_name; String prefix; + String disk; + /// If false, MergeTreePartInfo is in invalid state (directory name was not successfully parsed). bool valid_name; diff --git a/dbms/src/Storages/System/StorageSystemDetachedParts.cpp b/dbms/src/Storages/System/StorageSystemDetachedParts.cpp index b95a299af68..acda98203db 100644 --- a/dbms/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/dbms/src/Storages/System/StorageSystemDetachedParts.cpp @@ -35,6 +35,7 @@ protected: {"table", std::make_shared()}, {"partition_id", std::make_shared(std::make_shared())}, {"name", std::make_shared()}, + {"disk", std::make_shared()}, {"reason", std::make_shared(std::make_shared())}, {"min_block_number", std::make_shared(std::make_shared())}, {"max_block_number", std::make_shared(std::make_shared())}, @@ -66,6 +67,7 @@ protected: new_columns[i++]->insert(info.table); new_columns[i++]->insert(p.valid_name ? p.partition_id : Field()); new_columns[i++]->insert(p.dir_name); + new_columns[i++]->insert(p.disk); new_columns[i++]->insert(p.valid_name ? p.prefix : Field()); new_columns[i++]->insert(p.valid_name ? p.min_block : Field()); new_columns[i++]->insert(p.valid_name ? p.max_block : Field()); From 67d2703d30a66a362c81b292da056e1ea17ad005 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 19 Nov 2019 14:29:52 +0300 Subject: [PATCH 02/42] Fixed tests to support disk name in `system.detached_parts`. --- .../0_stateless/00502_custom_partitioning_local.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.reference b/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.reference index fff28819e74..7b14a2d4edc 100644 --- a/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.reference +++ b/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.reference @@ -9,7 +9,7 @@ Sum before DETACH PARTITION: Sum after DETACH PARTITION: 0 system.detached_parts after DETACH PARTITION: -default not_partitioned all all_1_2_1 1 2 1 +default not_partitioned all all_1_2_1 default 1 2 1 *** Partitioned by week *** Parts before OPTIMIZE: 1999-12-27 19991227_1_1_0 From 008e1efa753f4e22a1ef5d008ed29d3d98eb0c50 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 19 Nov 2019 14:43:33 +0300 Subject: [PATCH 03/42] Added test for disk name in `system.detached_parts`. --- .../integration/test_multiple_disks/test.py | 32 ++++++++++++++++++- 1 file changed, 31 insertions(+), 1 deletion(-) diff --git a/dbms/tests/integration/test_multiple_disks/test.py b/dbms/tests/integration/test_multiple_disks/test.py index 446eca88142..38ed618845f 100644 --- a/dbms/tests/integration/test_multiple_disks/test.py +++ b/dbms/tests/integration/test_multiple_disks/test.py @@ -768,12 +768,42 @@ def test_concurrent_alter_move_and_drop(start_cluster, name, engine): node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) +@pytest.mark.parametrize("name,engine", [ + ("detach_attach_mt","MergeTree()"), + ("replicated_detach_attach_mt","ReplicatedMergeTree('/clickhouse/replicated_detach_attach_mt', '1')",), +]) +def test_detach_attach(start_cluster, name, engine): + try: + node1.query(""" + CREATE TABLE {name} ( + s1 String + ) ENGINE = {engine} + ORDER BY tuple() + SETTINGS storage_policy='moving_jbod_with_external' + """.format(name=name, engine=engine)) + + data = [] # 5MB in total + for i in range(5): + data.append(get_random_string(1024 * 1024)) # 1MB row + node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data]))) + + node1.query("ALTER TABLE {} DETACH PARTITION tuple()".format(name)) + assert node1.query("SELECT count() FROM {}".format(name)).strip() == "0" + + assert node1.query("SELECT disk FROM system.detached_parts WHERE table = '{}'".format(name)).strip() == "jbod1" + + node1.query("ALTER TABLE {} ATTACH PARTITION tuple()".format(name)) + assert node1.query("SELECT count() FROM {}".format(name)).strip() == "5" + + finally: + node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) + + @pytest.mark.parametrize("name,engine", [ ("mutating_mt","MergeTree()"), ("replicated_mutating_mt","ReplicatedMergeTree('/clickhouse/replicated_mutating_mt', '1')",), ]) def test_mutate_to_another_disk(start_cluster, name, engine): - try: node1.query(""" CREATE TABLE {name} ( From 65ff10c81b894f9f8c7e7ce28e82fe68006ca5b9 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 20 Nov 2019 14:56:38 +0300 Subject: [PATCH 04/42] Use Minio for S3 Storage integration tests. --- dbms/tests/integration/README.md | 4 +- dbms/tests/integration/helpers/cluster.py | 190 ++++++--- .../helpers/docker_compose_minio.yml | 28 ++ dbms/tests/integration/image/Dockerfile | 2 +- .../integration/test_storage_s3/server.py | 368 ------------------ .../tests/integration/test_storage_s3/test.py | 223 ++++++----- 6 files changed, 274 insertions(+), 541 deletions(-) create mode 100644 dbms/tests/integration/helpers/docker_compose_minio.yml delete mode 100644 dbms/tests/integration/test_storage_s3/server.py diff --git a/dbms/tests/integration/README.md b/dbms/tests/integration/README.md index a0a5322e5a6..0db5ee258b2 100644 --- a/dbms/tests/integration/README.md +++ b/dbms/tests/integration/README.md @@ -14,9 +14,9 @@ Don't use Docker from your system repository. * [pip](https://pypi.python.org/pypi/pip) and `libpq-dev`. To install: `sudo apt-get install python-pip libpq-dev` * [py.test](https://docs.pytest.org/) testing framework. To install: `sudo -H pip install pytest` -* [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install docker-compose docker dicttoxml kazoo PyMySQL psycopg2 pymongo tzlocal kafka-python protobuf pytest-timeout` +* [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install docker-compose docker dicttoxml kazoo PyMySQL psycopg2 pymongo tzlocal kafka-python protobuf pytest-timeout minio` -(highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python-pytest python-dicttoxml python-docker python-pymysql python-pymongo python-tzlocal python-kazoo python-psycopg2 python-kafka python-pytest-timeout` +(highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python-pytest python-dicttoxml python-docker python-pymysql python-pymongo python-tzlocal python-kazoo python-psycopg2 python-kafka python-pytest-timeout python-minio` If you want to run the tests under a non-privileged user, you must add this user to `docker` group: `sudo usermod -aG docker $USER` and re-login. (You must close all your sessions (for example, restart your computer)) diff --git a/dbms/tests/integration/helpers/cluster.py b/dbms/tests/integration/helpers/cluster.py index 72bd070f8ec..16d5195439e 100644 --- a/dbms/tests/integration/helpers/cluster.py +++ b/dbms/tests/integration/helpers/cluster.py @@ -1,34 +1,33 @@ +import base64 +import distutils.dir_util +import errno import os import os.path as p import pwd import re -import subprocess import shutil -import distutils.dir_util import socket +import subprocess import time -import errno -from dicttoxml import dicttoxml -import pymysql +import urllib import xml.dom.minidom +import logging +import docker +import psycopg2 +import pymongo +import pymysql +from dicttoxml import dicttoxml from kazoo.client import KazooClient from kazoo.exceptions import KazooException -import psycopg2 -import requests -import base64 -import pymongo -import urllib +from minio import Minio -import docker -from docker.errors import ContainerError - -from .client import Client, CommandRequest +from .client import Client from .hdfs_api import HDFSApi - HELPERS_DIR = p.dirname(__file__) DEFAULT_ENV_NAME = 'env_file' + def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): full_path = os.path.join(path, fname) with open(full_path, 'w') as f: @@ -36,16 +35,19 @@ def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): f.write("=".join([var, value]) + "\n") return full_path + def subprocess_check_call(args): # Uncomment for debugging # print('run:', ' ' . join(args)) subprocess.check_call(args) + def subprocess_call(args): # Uncomment for debugging # print('run:', ' ' . join(args)) subprocess.call(args) + def get_odbc_bridge_path(): path = os.environ.get('CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH') if path is None: @@ -71,11 +73,15 @@ class ClickHouseCluster: self.base_dir = p.dirname(base_path) self.name = name if name is not None else '' - self.base_configs_dir = base_configs_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR', '/etc/clickhouse-server/') - self.server_bin_path = p.realpath(server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse')) + self.base_configs_dir = base_configs_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR', + '/etc/clickhouse-server/') + self.server_bin_path = p.realpath( + server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse')) self.odbc_bridge_bin_path = p.realpath(odbc_bridge_bin_path or get_odbc_bridge_path()) - self.client_bin_path = p.realpath(client_bin_path or os.environ.get('CLICKHOUSE_TESTS_CLIENT_BIN_PATH', '/usr/bin/clickhouse-client')) - self.zookeeper_config_path = p.join(self.base_dir, zookeeper_config_path) if zookeeper_config_path else p.join(HELPERS_DIR, 'zookeeper_config.xml') + self.client_bin_path = p.realpath( + client_bin_path or os.environ.get('CLICKHOUSE_TESTS_CLIENT_BIN_PATH', '/usr/bin/clickhouse-client')) + self.zookeeper_config_path = p.join(self.base_dir, zookeeper_config_path) if zookeeper_config_path else p.join( + HELPERS_DIR, 'zookeeper_config.xml') self.project_name = pwd.getpwuid(os.getuid()).pw_name + p.basename(self.base_dir) + self.name # docker-compose removes everything non-alphanumeric from project names so we do it too. @@ -105,17 +111,29 @@ class ClickHouseCluster: self.with_net_trics = False self.with_redis = False + self.with_minio = False + self.minio_host = "minio1" + self.minio_bucket = "root" + self.minio_port = 9001 + self.minio_client = None # type: Minio + self.minio_redirect_host = "redirect" + self.minio_redirect_port = 80 + self.docker_client = None self.is_up = False - def get_client_cmd(self): cmd = self.client_bin_path if p.basename(cmd) == 'clickhouse': cmd += " client" return cmd - def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, with_redis=False, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=[]): + def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, + with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, + with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, + with_redis=False, with_minio=False, + hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", + stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=[]): """Add an instance to the cluster. name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. @@ -133,9 +151,11 @@ class ClickHouseCluster: instance = ClickHouseInstance( self, self.base_dir, name, config_dir, main_configs, user_configs, macros, with_zookeeper, - self.zookeeper_config_path, with_mysql, with_kafka, with_mongo, with_redis, self.base_configs_dir, self.server_bin_path, + self.zookeeper_config_path, with_mysql, with_kafka, with_mongo, with_redis, with_minio, + self.base_configs_dir, self.server_bin_path, self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname, - env_variables=env_variables, image=image, stay_alive=stay_alive, ipv4_address=ipv4_address, ipv6_address=ipv6_address, + env_variables=env_variables, image=image, stay_alive=stay_alive, ipv4_address=ipv4_address, + ipv6_address=ipv6_address, with_installed_binary=with_installed_binary, tmpfs=tmpfs) self.instances[name] = instance @@ -150,14 +170,14 @@ class ClickHouseCluster: self.with_zookeeper = True self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_zookeeper.yml')]) self.base_zookeeper_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_zookeeper.yml')] + self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_zookeeper.yml')] cmds.append(self.base_zookeeper_cmd) if with_mysql and not self.with_mysql: self.with_mysql = True self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_mysql.yml')]) self.base_mysql_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_mysql.yml')] + self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_mysql.yml')] cmds.append(self.base_mysql_cmd) @@ -165,7 +185,7 @@ class ClickHouseCluster: self.with_postgres = True self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')]) self.base_postgres_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')] + self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')] cmds.append(self.base_postgres_cmd) if with_odbc_drivers and not self.with_odbc_drivers: @@ -181,28 +201,29 @@ class ClickHouseCluster: self.with_postgres = True self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')]) self.base_postgres_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')] + self.project_name, '--file', + p.join(HELPERS_DIR, 'docker_compose_postgres.yml')] cmds.append(self.base_postgres_cmd) if with_kafka and not self.with_kafka: self.with_kafka = True self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_kafka.yml')]) self.base_kafka_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_kafka.yml')] + self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_kafka.yml')] cmds.append(self.base_kafka_cmd) if with_hdfs and not self.with_hdfs: self.with_hdfs = True self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_hdfs.yml')]) self.base_hdfs_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_hdfs.yml')] + self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_hdfs.yml')] cmds.append(self.base_hdfs_cmd) if with_mongo and not self.with_mongo: self.with_mongo = True self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_mongo.yml')]) self.base_mongo_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_mongo.yml')] + self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_mongo.yml')] cmds.append(self.base_mongo_cmd) if self.with_net_trics: @@ -213,12 +234,17 @@ class ClickHouseCluster: self.with_redis = True self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_redis.yml')]) self.base_redis_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_redis.yml')] + self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_redis.yml')] + if with_minio and not self.with_minio: + self.with_minio = True + self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_minio.yml')]) + self.base_minio_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', + self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_minio.yml')] + cmds.append(self.base_minio_cmd) return instance - def get_instance_docker_id(self, instance_name): # According to how docker-compose names containers. return self.project_name + '_' + instance_name + '_1' @@ -246,7 +272,7 @@ class ClickHouseCluster: subprocess.check_call(self.base_cmd + ["up", "--force-recreate", "--no-deps", "-d", node.name]) node.ip_address = self.get_instance_ip(node.name) node.client = Client(node.ip_address, command=self.client_bin_path) - start_deadline = time.time() + 20.0 # seconds + start_deadline = time.time() + 20.0 # seconds node.wait_for_start(start_deadline) return node @@ -327,19 +353,39 @@ class ClickHouseCluster: print "Can't connect to Mongo " + str(ex) time.sleep(1) + def wait_minio_to_start(self, timeout=10): + minio_client = Minio('localhost:9001', + access_key='minio', + secret_key='minio123', + secure=False) + start = time.time() + while time.time() - start < timeout: + try: + buckets = minio_client.list_buckets() + self.minio_client = minio_client + logging.info("Connected to Minio %s", buckets) + return + except Exception as ex: + logging.warning("Can't connect to Minio: %s", str(ex)) + time.sleep(1) + def start(self, destroy_dirs=True): if self.is_up: return # Just in case kill unstopped containers from previous launch try: + logging.info("Trying to kill unstopped containers...") + if not subprocess_call(['docker-compose', 'kill']): subprocess_call(['docker-compose', 'down', '--volumes']) except: pass + logging.info("Unstopped containers killed") + if destroy_dirs and p.exists(self.instances_dir): - print "Removing instances dir", self.instances_dir + logging.info("Removing instances dir %s", self.instances_dir) shutil.rmtree(self.instances_dir) for instance in self.instances.values(): @@ -379,25 +425,36 @@ class ClickHouseCluster: subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate']) time.sleep(10) + if self.with_minio and self.base_minio_cmd: + minio_start_cmd = self.base_minio_cmd + common_opts + logging.info("Trying to create Minio instance by command %s", ' '.join(map(str, minio_start_cmd))) + subprocess_check_call(minio_start_cmd) + logging.info("Trying to connect to Minio...") + self.wait_minio_to_start() - subprocess_check_call(self.base_cmd + ['up', '-d', '--no-recreate']) + clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate'] + logging.info("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd))) + subprocess_check_call(clickhouse_start_cmd) + logging.info("ClickHouse instance created") - start_deadline = time.time() + 20.0 # seconds + start_deadline = time.time() + 20.0 # seconds for instance in self.instances.itervalues(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) + logging.info("Waiting for ClickHouse start...") instance.wait_for_start(start_deadline) + logging.info("ClickHouse started") instance.client = Client(instance.ip_address, command=self.client_bin_path) self.is_up = True - def shutdown(self, kill=True): if kill: subprocess_check_call(self.base_cmd + ['kill']) subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans']) + self.is_up = False self.docker_client = None @@ -407,18 +464,15 @@ class ClickHouseCluster: instance.ip_address = None instance.client = None - def open_bash_shell(self, instance_name): os.system(' '.join(self.base_cmd + ['exec', instance_name, '/bin/bash'])) - def get_kazoo_client(self, zoo_instance_name): zk = KazooClient(hosts=self.get_instance_ip(zoo_instance_name)) zk.start() return zk - - def run_kazoo_commands_with_retries(self, kazoo_callback, zoo_instance_name = 'zoo1', repeats=1, sleep_for=1): + def run_kazoo_commands_with_retries(self, kazoo_callback, zoo_instance_name='zoo1', repeats=1, sleep_for=1): for i in range(repeats - 1): try: kazoo_callback(self.get_kazoo_client(zoo_instance_name)) @@ -429,7 +483,6 @@ class ClickHouseCluster: kazoo_callback(self.get_kazoo_client(zoo_instance_name)) - def add_zookeeper_startup_command(self, command): self.pre_zookeeper_commands.append(command) @@ -472,8 +525,10 @@ class ClickHouseInstance: def __init__( self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macros, - with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_mongo, with_redis, base_configs_dir, server_bin_path, odbc_bridge_bin_path, - clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", + with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_mongo, with_redis, with_minio, + base_configs_dir, server_bin_path, odbc_bridge_bin_path, + clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables={}, + image="yandex/clickhouse-integration-test", stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=[]): self.name = name @@ -499,6 +554,7 @@ class ClickHouseInstance: self.with_kafka = with_kafka self.with_mongo = with_mongo self.with_redis = with_redis + self.with_minio = with_minio self.path = p.join(self.cluster.instances_dir, name) self.docker_compose_path = p.join(self.path, 'docker_compose.yml') @@ -512,7 +568,7 @@ class ClickHouseInstance: self.docker_client = None self.ip_address = None self.client = None - self.default_timeout = 20.0 # 20 sec + self.default_timeout = 20.0 # 20 sec self.image = image self.stay_alive = stay_alive self.ipv4_address = ipv4_address @@ -523,7 +579,8 @@ class ClickHouseInstance: def query(self, sql, stdin=None, timeout=None, settings=None, user=None, ignore_error=False): return self.client.query(sql, stdin, timeout, settings, user, ignore_error) - def query_with_retry(self, sql, stdin=None, timeout=None, settings=None, user=None, ignore_error=False, retry_count=20, sleep_time=0.5, check_callback=lambda x: True): + def query_with_retry(self, sql, stdin=None, timeout=None, settings=None, user=None, ignore_error=False, + retry_count=20, sleep_time=0.5, check_callback=lambda x: True): result = None for i in range(retry_count): try: @@ -553,7 +610,7 @@ class ClickHouseInstance: # Connects to the instance via HTTP interface, sends a query and returns the answer def http_query(self, sql, data=None): - return urllib.urlopen("http://"+self.ip_address+":8123/?query="+urllib.quote(sql,safe=''), data).read() + return urllib.urlopen("http://" + self.ip_address + ":8123/?query=" + urllib.quote(sql, safe=''), data).read() def restart_clickhouse(self, stop_start_wait_sec=5): if not self.stay_alive: @@ -575,17 +632,21 @@ class ClickHouseInstance: return output def contains_in_log(self, substring): - result = self.exec_in_container(["bash", "-c", "grep '{}' /var/log/clickhouse-server/clickhouse-server.log || true".format(substring)]) + result = self.exec_in_container( + ["bash", "-c", "grep '{}' /var/log/clickhouse-server/clickhouse-server.log || true".format(substring)]) return len(result) > 0 def copy_file_to_container(self, local_path, dest_path): with open(local_path, 'r') as fdata: data = fdata.read() encoded_data = base64.b64encode(data) - self.exec_in_container(["bash", "-c", "echo {} | base64 --decode > {}".format(encoded_data, dest_path)], user='root') + self.exec_in_container(["bash", "-c", "echo {} | base64 --decode > {}".format(encoded_data, dest_path)], + user='root') def get_process_pid(self, process_name): - output = self.exec_in_container(["bash", "-c", "ps ax | grep '{}' | grep -v 'grep' | grep -v 'bash -c' | awk '{{print $1}}'".format(process_name)]) + output = self.exec_in_container(["bash", "-c", + "ps ax | grep '{}' | grep -v 'grep' | grep -v 'bash -c' | awk '{{print $1}}'".format( + process_name)]) if output: try: pid = int(output.split('\n')[0].strip()) @@ -594,7 +655,6 @@ class ClickHouseInstance: return None return None - def restart_with_latest_version(self, stop_start_wait_sec=10, callback_onstop=None, signal=15): if not self.stay_alive: raise Exception("Cannot restart not stay alive container") @@ -610,8 +670,12 @@ class ClickHouseInstance: if callback_onstop: callback_onstop(self) - self.exec_in_container(["bash", "-c", "cp /usr/share/clickhouse_fresh /usr/bin/clickhouse && chmod 777 /usr/bin/clickhouse"], user='root') - self.exec_in_container(["bash", "-c", "cp /usr/share/clickhouse-odbc-bridge_fresh /usr/bin/clickhouse-odbc-bridge && chmod 777 /usr/bin/clickhouse"], user='root') + self.exec_in_container( + ["bash", "-c", "cp /usr/share/clickhouse_fresh /usr/bin/clickhouse && chmod 777 /usr/bin/clickhouse"], + user='root') + self.exec_in_container(["bash", "-c", + "cp /usr/share/clickhouse-odbc-bridge_fresh /usr/bin/clickhouse-odbc-bridge && chmod 777 /usr/bin/clickhouse"], + user='root') self.exec_in_container(["bash", "-c", "{} --daemon".format(CLICKHOUSE_START_COMMAND)], user=str(os.getuid())) from helpers.test_tools import assert_eq_with_retry # wait start @@ -620,15 +684,12 @@ class ClickHouseInstance: def get_docker_handle(self): return self.docker_client.containers.get(self.docker_id) - def stop(self): self.get_docker_handle().stop() - def start(self): self.get_docker_handle().start() - def wait_for_start(self, deadline=None, timeout=None): start_time = time.time() @@ -637,9 +698,11 @@ class ClickHouseInstance: while True: handle = self.get_docker_handle() - status = handle.status; + status = handle.status if status == 'exited': - raise Exception("Instance `{}' failed to start. Container status: {}, logs: {}".format(self.name, status, handle.logs())) + raise Exception( + "Instance `{}' failed to start. Container status: {}, logs: {}".format(self.name, status, + handle.logs())) current_time = time.time() time_left = deadline - current_time @@ -664,7 +727,6 @@ class ClickHouseInstance: finally: sock.close() - @staticmethod def dict_to_xml(dictionary): xml_str = dicttoxml(dictionary, custom_root="yandex", attr_type=False) @@ -676,7 +738,7 @@ class ClickHouseInstance: return { "SQLite3": { "DSN": "sqlite3_odbc", - "Database" : "/tmp/sqliteodbc", + "Database": "/tmp/sqliteodbc", "Driver": "/usr/lib/x86_64-linux-gnu/odbc/libsqlite3odbc.so", "Setup": "/usr/lib/x86_64-linux-gnu/odbc/libsqlite3odbc.so", }, @@ -750,7 +812,7 @@ class ClickHouseInstance: macros = self.macros.copy() macros['instance'] = self.name with open(p.join(config_d_dir, 'macros.xml'), 'w') as macros_config: - macros_config.write(self.dict_to_xml({"macros" : macros})) + macros_config.write(self.dict_to_xml({"macros": macros})) # Put ZooKeeper config if self.with_zookeeper: @@ -789,6 +851,10 @@ class ClickHouseInstance: depends_on.append("zoo2") depends_on.append("zoo3") + if self.with_minio: + depends_on.append("minio1") + depends_on.append("redirect") + env_file = _create_env_file(os.path.dirname(self.docker_compose_path), self.env_variables) odbc_ini_path = "" @@ -820,7 +886,6 @@ class ClickHouseInstance: binary_volume = "- " + self.server_bin_path + ":/usr/share/clickhouse_fresh" odbc_bridge_volume = "- " + self.odbc_bridge_bin_path + ":/usr/share/clickhouse-odbc-bridge_fresh" - with open(self.docker_compose_path, 'w') as docker_compose: docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format( image=self.image, @@ -844,7 +909,6 @@ class ClickHouseInstance: ipv6_address=ipv6_address, )) - def destroy_dir(self): if p.exists(self.path): shutil.rmtree(self.path) diff --git a/dbms/tests/integration/helpers/docker_compose_minio.yml b/dbms/tests/integration/helpers/docker_compose_minio.yml new file mode 100644 index 00000000000..7e024d1eb05 --- /dev/null +++ b/dbms/tests/integration/helpers/docker_compose_minio.yml @@ -0,0 +1,28 @@ +version: '2.2' + +services: + minio1: + image: minio/minio + volumes: + - data1-1:/data1 + ports: + - "9001:9001" + environment: + MINIO_ACCESS_KEY: minio + MINIO_SECRET_KEY: minio123 + command: server --address :9001 /data1-1 + healthcheck: + test: ["CMD", "curl", "-f", "http://localhost:9001/minio/health/live"] + interval: 30s + timeout: 20s + retries: 3 + +# Redirects all requests to origin Minio. + redirect: + image: schmunk42/nginx-redirect + environment: + - SERVER_REDIRECT=minio1:9001 + - SERVER_REDIRECT_CODE=307 + +volumes: + data1-1: diff --git a/dbms/tests/integration/image/Dockerfile b/dbms/tests/integration/image/Dockerfile index 0ebecb43681..ec66be91a06 100644 --- a/dbms/tests/integration/image/Dockerfile +++ b/dbms/tests/integration/image/Dockerfile @@ -34,7 +34,7 @@ RUN apt-get update \ ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone -RUN pip install pytest docker-compose==1.22.0 docker dicttoxml kazoo PyMySQL psycopg2==2.7.5 pymongo tzlocal kafka-python protobuf redis aerospike pytest-timeout +RUN pip install urllib3==1.23 pytest docker-compose==1.22.0 docker dicttoxml kazoo PyMySQL psycopg2==2.7.5 pymongo tzlocal kafka-python protobuf redis aerospike pytest-timeout minio ENV DOCKER_CHANNEL stable ENV DOCKER_VERSION 17.09.1-ce diff --git a/dbms/tests/integration/test_storage_s3/server.py b/dbms/tests/integration/test_storage_s3/server.py deleted file mode 100644 index 712af20759d..00000000000 --- a/dbms/tests/integration/test_storage_s3/server.py +++ /dev/null @@ -1,368 +0,0 @@ -try: - from BaseHTTPServer import BaseHTTPRequestHandler -except ImportError: - from http.server import BaseHTTPRequestHandler - -try: - from BaseHTTPServer import HTTPServer -except ImportError: - from http.server import HTTPServer - -try: - import urllib.parse as urlparse -except ImportError: - import urlparse - -import json -import logging -import os -import socket -import sys -import threading -import time -import uuid -import xml.etree.ElementTree - -BASE_DIR = os.path.dirname(__file__) - -logging.getLogger().setLevel(logging.INFO) -file_handler = logging.FileHandler(os.path.join(BASE_DIR, "test-server.log"), "a", encoding="utf-8") -file_handler.setFormatter(logging.Formatter("%(asctime)s %(message)s")) -logging.getLogger().addHandler(file_handler) -logging.getLogger().addHandler(logging.StreamHandler()) - -communication_port = int(sys.argv[1]) -bucket = sys.argv[2] - - -def GetFreeTCPPortsAndIP(n): - result = [] - sockets = [] - for i in range(n): - tcp = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - tcp.bind((socket.gethostname(), 0)) - addr, port = tcp.getsockname() - result.append(port) - sockets.append(tcp) - [ s.close() for s in sockets ] - return result, addr - -( - redirecting_to_http_port, - simple_server_port, - preserving_data_port, - multipart_preserving_data_port, - redirecting_preserving_data_port -), localhost = GetFreeTCPPortsAndIP(5) - - -data = { - "redirecting_to_http_port": redirecting_to_http_port, - "preserving_data_port": preserving_data_port, - "multipart_preserving_data_port": multipart_preserving_data_port, - "redirecting_preserving_data_port": redirecting_preserving_data_port, -} - - -class SimpleHTTPServerHandler(BaseHTTPRequestHandler): - def do_GET(self): - logging.info("GET {}".format(self.path)) - if self.path == "/milovidov/test.csv": - self.send_response(200) - self.send_header("Content-type", "text/plain") - self.end_headers() - data["redirect_csv_data"] = [[42, 87, 44], [55, 33, 81], [1, 0, 9]] - self.wfile.write("".join([ "{},{},{}\n".format(*row) for row in data["redirect_csv_data"]])) - else: - self.send_response(404) - self.end_headers() - self.finish() - - -class RedirectingToHTTPHandler(BaseHTTPRequestHandler): - def do_GET(self): - self.send_response(307) - self.send_header("Content-type", "text/xml") - self.send_header("Location", "http://{}:{}/milovidov/test.csv".format(localhost, simple_server_port)) - self.end_headers() - self.wfile.write(r""" - - TemporaryRedirect - Please re-send this request to the specified temporary endpoint. - Continue to use the original request endpoint for future requests. - storage.yandexcloud.net -""".encode()) - self.finish() - - -class PreservingDataHandler(BaseHTTPRequestHandler): - protocol_version = "HTTP/1.1" - - def parse_request(self): - result = BaseHTTPRequestHandler.parse_request(self) - # Adaptation to Python 3. - if sys.version_info.major == 2 and result == True: - expect = self.headers.get("Expect", "") - if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"): - if not self.handle_expect_100(): - return False - return result - - def send_response_only(self, code, message=None): - if message is None: - if code in self.responses: - message = self.responses[code][0] - else: - message = "" - if self.request_version != "HTTP/0.9": - self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message)) - - def handle_expect_100(self): - logging.info("Received Expect-100") - self.send_response_only(100) - self.end_headers() - return True - - def do_POST(self): - self.send_response(200) - query = urlparse.urlparse(self.path).query - logging.info("PreservingDataHandler POST ?" + query) - if query == "uploads": - post_data = r""" -TEST""".encode() - self.send_header("Content-length", str(len(post_data))) - self.send_header("Content-type", "text/plain") - self.end_headers() - self.wfile.write(post_data) - else: - post_data = self.rfile.read(int(self.headers.get("Content-Length"))) - self.send_header("Content-type", "text/plain") - self.end_headers() - data["received_data_completed"] = True - data["finalize_data"] = post_data - data["finalize_data_query"] = query - self.finish() - - def do_PUT(self): - self.send_response(200) - self.send_header("Content-type", "text/plain") - self.send_header("ETag", "hello-etag") - self.end_headers() - query = urlparse.urlparse(self.path).query - path = urlparse.urlparse(self.path).path - logging.info("Content-Length = " + self.headers.get("Content-Length")) - logging.info("PUT " + query) - assert self.headers.get("Content-Length") - assert self.headers["Expect"] == "100-continue" - put_data = self.rfile.read() - data.setdefault("received_data", []).append(put_data) - logging.info("PUT to {}".format(path)) - self.server.storage[path] = put_data - self.finish() - - def do_GET(self): - path = urlparse.urlparse(self.path).path - if path in self.server.storage: - self.send_response(200) - self.send_header("Content-type", "text/plain") - self.send_header("Content-length", str(len(self.server.storage[path]))) - self.end_headers() - self.wfile.write(self.server.storage[path]) - else: - self.send_response(404) - self.end_headers() - self.finish() - - -class MultipartPreservingDataHandler(BaseHTTPRequestHandler): - protocol_version = "HTTP/1.1" - - def parse_request(self): - result = BaseHTTPRequestHandler.parse_request(self) - # Adaptation to Python 3. - if sys.version_info.major == 2 and result == True: - expect = self.headers.get("Expect", "") - if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"): - if not self.handle_expect_100(): - return False - return result - - def send_response_only(self, code, message=None): - if message is None: - if code in self.responses: - message = self.responses[code][0] - else: - message = "" - if self.request_version != "HTTP/0.9": - self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message)) - - def handle_expect_100(self): - logging.info("Received Expect-100") - self.send_response_only(100) - self.end_headers() - return True - - def do_POST(self): - query = urlparse.urlparse(self.path).query - logging.info("MultipartPreservingDataHandler POST ?" + query) - if query == "uploads": - self.send_response(200) - post_data = r""" -TEST""".encode() - self.send_header("Content-length", str(len(post_data))) - self.send_header("Content-type", "text/plain") - self.end_headers() - self.wfile.write(post_data) - else: - try: - assert query == "uploadId=TEST" - logging.info("Content-Length = " + self.headers.get("Content-Length")) - post_data = self.rfile.read(int(self.headers.get("Content-Length"))) - root = xml.etree.ElementTree.fromstring(post_data) - assert root.tag == "CompleteMultipartUpload" - assert len(root) > 1 - content = "" - for i, part in enumerate(root): - assert part.tag == "Part" - assert len(part) == 2 - assert part[0].tag == "PartNumber" - assert part[1].tag == "ETag" - assert int(part[0].text) == i + 1 - content += self.server.storage["@"+part[1].text] - data.setdefault("multipart_received_data", []).append(content) - data["multipart_parts"] = len(root) - self.send_response(200) - self.send_header("Content-type", "text/plain") - self.end_headers() - logging.info("Sending 200") - except: - logging.error("Sending 500") - self.send_response(500) - self.finish() - - def do_PUT(self): - uid = uuid.uuid4() - self.send_response(200) - self.send_header("Content-type", "text/plain") - self.send_header("ETag", str(uid)) - self.end_headers() - query = urlparse.urlparse(self.path).query - path = urlparse.urlparse(self.path).path - logging.info("Content-Length = " + self.headers.get("Content-Length")) - logging.info("PUT " + query) - assert self.headers.get("Content-Length") - assert self.headers["Expect"] == "100-continue" - put_data = self.rfile.read() - data.setdefault("received_data", []).append(put_data) - logging.info("PUT to {}".format(path)) - self.server.storage["@"+str(uid)] = put_data - self.finish() - - def do_GET(self): - path = urlparse.urlparse(self.path).path - if path in self.server.storage: - self.send_response(200) - self.send_header("Content-type", "text/plain") - self.send_header("Content-length", str(len(self.server.storage[path]))) - self.end_headers() - self.wfile.write(self.server.storage[path]) - else: - self.send_response(404) - self.end_headers() - self.finish() - - -class RedirectingPreservingDataHandler(BaseHTTPRequestHandler): - protocol_version = "HTTP/1.1" - - def parse_request(self): - result = BaseHTTPRequestHandler.parse_request(self) - # Adaptation to Python 3. - if sys.version_info.major == 2 and result == True: - expect = self.headers.get("Expect", "") - if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"): - if not self.handle_expect_100(): - return False - return result - - def send_response_only(self, code, message=None): - if message is None: - if code in self.responses: - message = self.responses[code][0] - else: - message = "" - if self.request_version != "HTTP/0.9": - self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message)) - - def handle_expect_100(self): - logging.info("Received Expect-100") - return True - - def do_POST(self): - query = urlparse.urlparse(self.path).query - if query: - query = "?{}".format(query) - self.send_response(307) - self.send_header("Content-type", "text/xml") - self.send_header("Location", "http://{host}:{port}/{bucket}/test.csv{query}".format(host=localhost, port=preserving_data_port, bucket=bucket, query=query)) - self.end_headers() - self.wfile.write(r""" - - TemporaryRedirect - Please re-send this request to the specified temporary endpoint. - Continue to use the original request endpoint for future requests. - {host}:{port} -""".format(host=localhost, port=preserving_data_port).encode()) - self.finish() - - def do_PUT(self): - query = urlparse.urlparse(self.path).query - if query: - query = "?{}".format(query) - self.send_response(307) - self.send_header("Content-type", "text/xml") - self.send_header("Location", "http://{host}:{port}/{bucket}/test.csv{query}".format(host=localhost, port=preserving_data_port, bucket=bucket, query=query)) - self.end_headers() - self.wfile.write(r""" - - TemporaryRedirect - Please re-send this request to the specified temporary endpoint. - Continue to use the original request endpoint for future requests. - {host}:{port} -""".format(host=localhost, port=preserving_data_port).encode()) - self.finish() - - -class CommunicationServerHandler(BaseHTTPRequestHandler): - def do_GET(self): - self.send_response(200) - self.end_headers() - self.wfile.write(json.dumps(data)) - self.finish() - - def do_PUT(self): - self.send_response(200) - self.end_headers() - logging.info(self.rfile.read()) - self.finish() - - -servers = [] -servers.append(HTTPServer((localhost, communication_port), CommunicationServerHandler)) -servers.append(HTTPServer((localhost, redirecting_to_http_port), RedirectingToHTTPHandler)) -servers.append(HTTPServer((localhost, preserving_data_port), PreservingDataHandler)) -servers[-1].storage = {} -servers.append(HTTPServer((localhost, multipart_preserving_data_port), MultipartPreservingDataHandler)) -servers[-1].storage = {} -servers.append(HTTPServer((localhost, simple_server_port), SimpleHTTPServerHandler)) -servers.append(HTTPServer((localhost, redirecting_preserving_data_port), RedirectingPreservingDataHandler)) -jobs = [ threading.Thread(target=server.serve_forever) for server in servers ] -[ job.start() for job in jobs ] - -time.sleep(60) # Timeout - -logging.info("Shutting down") -[ server.shutdown() for server in servers ] -logging.info("Joining threads") -[ job.join() for job in jobs ] -logging.info("Done") diff --git a/dbms/tests/integration/test_storage_s3/test.py b/dbms/tests/integration/test_storage_s3/test.py index 212a54260c2..7afe55ce50b 100644 --- a/dbms/tests/integration/test_storage_s3/test.py +++ b/dbms/tests/integration/test_storage_s3/test.py @@ -1,159 +1,168 @@ -import httplib import json import logging -import os -import time -import traceback import pytest -from helpers.cluster import ClickHouseCluster - +from helpers.cluster import ClickHouseCluster, ClickHouseInstance logging.getLogger().setLevel(logging.INFO) logging.getLogger().addHandler(logging.StreamHandler()) -def get_communication_data(started_cluster): - conn = httplib.HTTPConnection(started_cluster.instances["dummy"].ip_address, started_cluster.communication_port) - conn.request("GET", "/") - r = conn.getresponse() - raw_data = r.read() - conn.close() - return json.loads(raw_data) +# Creates S3 bucket for tests and allows anonymous read-write access to it. +def prepare_s3_bucket(cluster): + minio_client = cluster.minio_client + + if minio_client.bucket_exists(cluster.minio_bucket): + minio_client.remove_bucket(cluster.minio_bucket) + + minio_client.make_bucket(cluster.minio_bucket) + + # Allows read-write access for bucket without authorization. + bucket_read_write_policy = {"Version": "2012-10-17", + "Statement": [ + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetBucketLocation", + "Resource": "arn:aws:s3:::root" + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:ListBucket", + "Resource": "arn:aws:s3:::root" + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetObject", + "Resource": "arn:aws:s3:::root/*" + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:PutObject", + "Resource": "arn:aws:s3:::root/*" + } + ]} + + minio_client.set_bucket_policy(cluster.minio_bucket, json.dumps(bucket_read_write_policy)) -def put_communication_data(started_cluster, body): - conn = httplib.HTTPConnection(started_cluster.instances["dummy"].ip_address, started_cluster.communication_port) - conn.request("PUT", "/", body) - r = conn.getresponse() - conn.close() +# Returns content of given S3 file as string. +def get_s3_file_content(cluster, filename): + # type: (ClickHouseCluster, str) -> str + + data = cluster.minio_client.get_object(cluster.minio_bucket, filename) + data_str = "" + for chunk in data.stream(): + data_str += chunk + return data_str @pytest.fixture(scope="module") -def started_cluster(): +def cluster(): try: cluster = ClickHouseCluster(__file__) - instance = cluster.add_instance("dummy") + cluster.add_instance("dummy", with_minio=True) + logging.info("Starting cluster...") cluster.start() + logging.info("Cluster started") - cluster.communication_port = 10000 - instance.copy_file_to_container(os.path.join(os.path.dirname(__file__), "server.py"), "test_server.py") - cluster.bucket = "abc" - instance.exec_in_container(["python", "test_server.py", str(cluster.communication_port), cluster.bucket], detach=True) - cluster.mock_host = instance.ip_address - - for i in range(10): - try: - data = get_communication_data(cluster) - cluster.redirecting_to_http_port = data["redirecting_to_http_port"] - cluster.preserving_data_port = data["preserving_data_port"] - cluster.multipart_preserving_data_port = data["multipart_preserving_data_port"] - cluster.redirecting_preserving_data_port = data["redirecting_preserving_data_port"] - except: - logging.error(traceback.format_exc()) - time.sleep(0.5) - else: - break - else: - assert False, "Could not initialize mock server" + prepare_s3_bucket(cluster) + logging.info("S3 bucket created") yield cluster - finally: cluster.shutdown() def run_query(instance, query, stdin=None, settings=None): + # type: (ClickHouseInstance, str, object, dict) -> str + logging.info("Running query '{}'...".format(query)) result = instance.query(query, stdin=stdin, settings=settings) logging.info("Query finished") + return result -def test_get_with_redirect(started_cluster): - instance = started_cluster.instances["dummy"] - format = "column1 UInt32, column2 UInt32, column3 UInt32" +# Test simple put. +def test_put(cluster): + # type: (ClickHouseCluster) -> None - put_communication_data(started_cluster, "=== Get with redirect test ===") - query = "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(started_cluster.mock_host, started_cluster.redirecting_to_http_port, format) - stdout = run_query(instance, query) - data = get_communication_data(started_cluster) - expected = [ [str(row[0]), str(row[1]), str(row[2]), str(row[0]*row[1]*row[2])] for row in data["redirect_csv_data"] ] - assert list(map(str.split, stdout.splitlines())) == expected - - -def test_put(started_cluster): - instance = started_cluster.instances["dummy"] - format = "column1 UInt32, column2 UInt32, column3 UInt32" - - logging.info("Phase 3") - put_communication_data(started_cluster, "=== Put test ===") + instance = cluster.instances["dummy"] # type: ClickHouseInstance + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" - put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(started_cluster.mock_host, started_cluster.preserving_data_port, started_cluster.bucket, format, values) + values_csv = "1,2,3\n3,2,1\n78,43,45\n" + filename = "test.csv" + put_query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( + cluster.minio_host, cluster.minio_port, cluster.minio_bucket, filename, table_format, values) run_query(instance, put_query) - data = get_communication_data(started_cluster) - received_data_completed = data["received_data_completed"] - received_data = data["received_data"] - finalize_data = data["finalize_data"] - finalize_data_query = data["finalize_data_query"] - assert received_data[-1].decode() == "1,2,3\n3,2,1\n78,43,45\n" - assert received_data_completed - assert finalize_data == "1hello-etag" - assert finalize_data_query == "uploadId=TEST" - -def test_put_csv(started_cluster): - instance = started_cluster.instances["dummy"] - format = "column1 UInt32, column2 UInt32, column3 UInt32" + assert values_csv == get_s3_file_content(cluster, filename) - put_communication_data(started_cluster, "=== Put test CSV ===") - put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') format CSV".format(started_cluster.mock_host, started_cluster.preserving_data_port, started_cluster.bucket, format) + +# Test put values in CSV format. +def test_put_csv(cluster): + # type: (ClickHouseCluster) -> None + + instance = cluster.instances["dummy"] # type: ClickHouseInstance + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + filename = "test.csv" + put_query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') format CSV".format( + cluster.minio_host, cluster.minio_port, cluster.minio_bucket, filename, table_format) csv_data = "8,9,16\n11,18,13\n22,14,2\n" run_query(instance, put_query, stdin=csv_data) - data = get_communication_data(started_cluster) - received_data_completed = data["received_data_completed"] - received_data = data["received_data"] - finalize_data = data["finalize_data"] - finalize_data_query = data["finalize_data_query"] - assert received_data[-1].decode() == csv_data - assert received_data_completed - assert finalize_data == "1hello-etag" - assert finalize_data_query == "uploadId=TEST" - -def test_put_with_redirect(started_cluster): - instance = started_cluster.instances["dummy"] - format = "column1 UInt32, column2 UInt32, column3 UInt32" + assert csv_data == get_s3_file_content(cluster, filename) - put_communication_data(started_cluster, "=== Put with redirect test ===") - other_values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" - query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(started_cluster.mock_host, started_cluster.redirecting_preserving_data_port, started_cluster.bucket, format, other_values) + +# Test put and get with S3 server redirect. +def test_put_get_with_redirect(cluster): + # type: (ClickHouseCluster) -> None + + instance = cluster.instances["dummy"] # type: ClickHouseInstance + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" + values_csv = "1,1,1\n1,1,1\n11,11,11\n" + filename = "test.csv" + query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( + cluster.minio_redirect_host, cluster.minio_redirect_port, cluster.minio_bucket, filename, table_format, values) run_query(instance, query) - query = "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(started_cluster.mock_host, started_cluster.preserving_data_port, started_cluster.bucket, format) + assert values_csv == get_s3_file_content(cluster, filename) + + query = "select *, column1*column2*column3 from s3('http://{}:{}/{}/{}', 'CSV', '{}')".format( + cluster.minio_redirect_host, cluster.minio_redirect_port, cluster.minio_bucket, filename, table_format) stdout = run_query(instance, query) + assert list(map(str.split, stdout.splitlines())) == [ ["1", "1", "1", "1"], ["1", "1", "1", "1"], ["11", "11", "11", "1331"], ] - data = get_communication_data(started_cluster) - received_data = data["received_data"] - assert received_data[-1].decode() == "1,1,1\n1,1,1\n11,11,11\n" -def test_multipart_put(started_cluster): - instance = started_cluster.instances["dummy"] - format = "column1 UInt32, column2 UInt32, column3 UInt32" +# Test multipart put. +def test_multipart_put(cluster): + # type: (ClickHouseCluster) -> None - put_communication_data(started_cluster, "=== Multipart test ===") - long_data = [[i, i+1, i+2] for i in range(100000)] - long_values = "".join([ "{},{},{}\n".format(x,y,z) for x, y, z in long_data ]) - put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') format CSV".format(started_cluster.mock_host, started_cluster.multipart_preserving_data_port, started_cluster.bucket, format) - run_query(instance, put_query, stdin=long_values, settings={'s3_min_upload_part_size': 1000000}) - data = get_communication_data(started_cluster) - assert "multipart_received_data" in data - received_data = data["multipart_received_data"] - assert received_data[-1].decode() == "".join([ "{},{},{}\n".format(x, y, z) for x, y, z in long_data ]) - assert 1 < data["multipart_parts"] < 10000 + instance = cluster.instances["dummy"] # type: ClickHouseInstance + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + long_data = [[i, i + 1, i + 2] for i in range(100000)] + long_values_csv = "".join(["{},{},{}\n".format(x, y, z) for x, y, z in long_data]) + filename = "test.csv" + put_query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') format CSV".format( + cluster.minio_host, cluster.minio_port, cluster.minio_bucket, filename, table_format) + + # Minimum size of part is 5 Mb for Minio. + # See: https://github.com/minio/minio/blob/master/docs/minio-limits.md + run_query(instance, put_query, stdin=long_values_csv, settings={'s3_min_upload_part_size': 5 * 1024 * 1024}) + + assert long_values_csv == get_s3_file_content(cluster, filename) From 586f60d2a08d0154628c1bd5ee564c135928da48 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 21 Nov 2019 16:13:38 +0300 Subject: [PATCH 05/42] Ensure multipart upload works in S3 storage tests. --- .../helpers/docker_compose_minio.yml | 3 ++ .../tests/integration/test_storage_s3/test.py | 37 +++++++++++++++---- 2 files changed, 33 insertions(+), 7 deletions(-) diff --git a/dbms/tests/integration/helpers/docker_compose_minio.yml b/dbms/tests/integration/helpers/docker_compose_minio.yml index 7e024d1eb05..41816427794 100644 --- a/dbms/tests/integration/helpers/docker_compose_minio.yml +++ b/dbms/tests/integration/helpers/docker_compose_minio.yml @@ -20,9 +20,12 @@ services: # Redirects all requests to origin Minio. redirect: image: schmunk42/nginx-redirect + volumes: + - /nginx:/nginx environment: - SERVER_REDIRECT=minio1:9001 - SERVER_REDIRECT_CODE=307 + - SERVER_ACCESS_LOG=/nginx/access.log volumes: data1-1: diff --git a/dbms/tests/integration/test_storage_s3/test.py b/dbms/tests/integration/test_storage_s3/test.py index 7afe55ce50b..1db472e3019 100644 --- a/dbms/tests/integration/test_storage_s3/test.py +++ b/dbms/tests/integration/test_storage_s3/test.py @@ -65,6 +65,14 @@ def get_s3_file_content(cluster, filename): return data_str +# Returns nginx access log lines. +def get_nginx_access_logs(): + handle = open("/nginx/access.log", "r") + data = handle.readlines() + handle.close() + return data + + @pytest.fixture(scope="module") def cluster(): try: @@ -155,14 +163,29 @@ def test_multipart_put(cluster): instance = cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - long_data = [[i, i + 1, i + 2] for i in range(100000)] - long_values_csv = "".join(["{},{},{}\n".format(x, y, z) for x, y, z in long_data]) - filename = "test.csv" - put_query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') format CSV".format( - cluster.minio_host, cluster.minio_port, cluster.minio_bucket, filename, table_format) # Minimum size of part is 5 Mb for Minio. # See: https://github.com/minio/minio/blob/master/docs/minio-limits.md - run_query(instance, put_query, stdin=long_values_csv, settings={'s3_min_upload_part_size': 5 * 1024 * 1024}) + min_part_size_bytes = 5 * 1024 * 1024 + csv_size_bytes = int(min_part_size_bytes * 1.5) # To have 2 parts. - assert long_values_csv == get_s3_file_content(cluster, filename) + one_line_length = 6 # 3 digits, 2 commas, 1 line separator. + + # Generate data having size more than one part + int_data = [[1, 2, 3] for i in range(csv_size_bytes / one_line_length)] + csv_data = "".join(["{},{},{}\n".format(x, y, z) for x, y, z in int_data]) + + assert len(csv_data) > min_part_size_bytes + + filename = "test_multipart.csv" + put_query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') format CSV".format( + cluster.minio_redirect_host, cluster.minio_redirect_port, cluster.minio_bucket, filename, table_format) + + run_query(instance, put_query, stdin=csv_data, settings={'s3_min_upload_part_size': min_part_size_bytes}) + + # Use Nginx access logs to count number of parts uploaded to Minio. + nginx_logs = get_nginx_access_logs() + uploaded_parts = filter(lambda log_line: log_line.find(filename) >= 0 and log_line.find("PUT") >= 0, nginx_logs) + assert uploaded_parts > 1 + + assert csv_data == get_s3_file_content(cluster, filename) From 6f3791f507b880602377b1812bf4a3e5c5591881 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Thu, 21 Nov 2019 18:03:19 +0300 Subject: [PATCH 06/42] Moved developer instruction --- .../instructions/developer_instruction_en.md | 269 +----------------- .../instructions/developer_instruction_ru.md | 269 +----------------- docs/en/development/developer_instruction.md | 268 +++++++++++++++++ docs/ru/development/developer_instruction.md | 268 +++++++++++++++++ 4 files changed, 538 insertions(+), 536 deletions(-) mode change 100644 => 120000 dbms/tests/instructions/developer_instruction_en.md mode change 100644 => 120000 dbms/tests/instructions/developer_instruction_ru.md create mode 100644 docs/en/development/developer_instruction.md create mode 100644 docs/ru/development/developer_instruction.md diff --git a/dbms/tests/instructions/developer_instruction_en.md b/dbms/tests/instructions/developer_instruction_en.md deleted file mode 100644 index 463f186fb3b..00000000000 --- a/dbms/tests/instructions/developer_instruction_en.md +++ /dev/null @@ -1,268 +0,0 @@ -Building of ClickHouse is supported on Linux, FreeBSD and Mac OS X. - -# If you use Windows - -If you use Windows, you need to create a virtual machine with Ubuntu. To start working with a virtual machine please install VirtualBox. You can download Ubuntu from the website: https://www.ubuntu.com/#download. Please create a virtual machine from the downloaded image (you should reserve at least 4GB of RAM for it). To run a command line terminal in Ubuntu, please locate a program containing the word "terminal" in its name (gnome-terminal, konsole etc.) or just press Ctrl+Alt+T. - - -# Creating a repository on GitHub - -To start working with ClickHouse repository you will need a GitHub account. - -You probably already have one, but if you don't, please register at https://github.com. In case you do not have SSH keys, you should generate them and then upload them on GitHub. It is required for sending over your patches. It is also possible to use the same SSH keys that you use with any other SSH servers - probably you already have those. - -Create a fork of ClickHouse repository. To do that please click on the "fork" button in the upper right corner at https://github.com/ClickHouse/ClickHouse. It will fork your own copy of ClickHouse/ClickHouse to your account. - -Development process consists of first committing the intended changes into your fork of ClickHouse and then creating a "pull request" for these changes to be accepted into the main repository (ClickHouse/ClickHouse). - -To work with git repositories, please install `git`. - -To do that in Ubuntu you would run in the command line terminal: -``` -sudo apt update -sudo apt install git -``` - -A brief manual on using Git can be found here: https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf. -For a detailed manual on Git see: https://git-scm.com/book/ru/v2. - - -# Cloning a repository to your development machine - -Next, you need to download the source files onto your working machine. This is called "to clone a repository" because it creates a local copy of the repository on your working machine. - -In the command line terminal run: -``` -git clone --recursive git@guthub.com:your_github_username/ClickHouse.git -cd ClickHouse -``` -Note: please, substitute *your_github_username* with what is appropriate! - -This command will create a directory `ClickHouse` containing the working copy of the project. - -It is important that the path to the working directory contains no whitespaces as it may lead to problems with running the build system. - -Please note that ClickHouse repository uses `submodules`. That is what the references to additional repositories are called (i.e. external libraries on which the project depends). It means that when cloning the repository you need to specify the `--recursive` flag as in the example above. If the repository has been cloned without submodules, to download them you need to run the following: -``` -git submodule init -git submodule update -``` -You can check status with command: `git submodule status`. - -If you get the following error message: -``` -Permission denied (publickey). -fatal: Could not read from remote repository. - -Please make sure you have the correct access rights -and the repository exists. -``` -It generally means that the SSH keys for connecting to GitHub are missing. These keys are normally located in `~/.ssh`. For SSH keys to be accepted you need to upload them in the settings section of GitHub UI. - -You can also clone the repository via https protocol: -``` -git clone https://github.com/ClickHouse/ClickHouse.git -``` -This however will not let you send your changes to the server. You can still use it temporarily and add the SSH keys later replacing the remote address of the repository with `git remote` command. - -You can also add original ClickHouse repo's address to your local repository to pull updates from there: -``` -git remote add upstream git@github.com:ClickHouse/ClickHouse.git -``` -After successfully running this command you will be able to pull updates from the main ClickHouse repo by running `git pull upstream master`. - - -# Build System - -ClickHouse uses CMake and Ninja for building. - -CMake - a meta-build system that can generate Ninja files (build tasks). -Ninja - a smaller build system with focus on speed used to execute those cmake generated tasks. - -To install on Ubuntu, Debian or Mint run `sudo apt install cmake ninja-build`. - -On CentOS, RedHat run `sudo yum install cmake ninja-build`. - -If you use Arch or Gentoo, you probably know it yourself how to install CMake. - -For installing CMake and Ninja on Mac OS X first install Homebrew and then install everything else via brew: -``` -/usr/bin/ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/install)" -brew install cmake ninja -``` - -Next, check the version of CMake: `cmake --version`. If it is below 3.3, you should install a newer version from the website: https://cmake.org/download/. - - -# Optional External Libraries - -ClickHouse uses several external libraries for building. Most of them do not need to be installed separately as they are built together with ClickHouse from the sources located in the submodules. You can check the list in `contrib`. - -There is a couple of libraries that are not built from sources but are supplied by the system: ICU and Readline, and thus are recommended to be installed. - -Ubuntu: `sudo apt install libicu-dev libreadline-dev` - -Mac OS X: `brew install icu4c readline` - -However, these libraries are optional and ClickHouse can well be built without them. ICU is used for support of `COLLATE` in `ORDER BY` (i.e. for sorting in turkish alphabet). Readline is used for more convenient command input in clickhouse-client. - - -# C++ Compiler - -Compilers GCC starting from version 9 and Clang version 8 or above are supported for building ClickHouse. - -Official Yandex builds currently use GCC because it generates machine code of slightly better performance (yielding a difference of up to several percent according to our benchmarks). And Clang is more convenient for development usually. Though, our continuous integration (CI) platform runs checks for about a dozen of build combinations. - -To install GCC on Ubuntu run: `sudo apt install gcc g++` - -Check the version of gcc: `gcc --version`. If it is below 9, then follow the instruction here: https://clickhouse.yandex/docs/en/development/build/#install-gcc-9. - -To install GCC on Mac OS X run: `brew install gcc`. - -If you decide to use Clang, you can also install `libc++` and `lld`, if you know what it is. Using `ccache` is also recommended. - - -# The Building process - -Now that you are ready to build ClickHouse we recommend you to create a separate directory `build` inside `ClickHouse` that will contain all of the build artefacts: -``` -mkdir build -cd build -``` -You can have several different directories (build_release, build_debug, etc.) for different types of build. - -While inside the `build` directory, configure your build by running CMake. Before the first run you need to define environment variables that specify compiler (version 9 gcc compiler in this example). -``` -export CC=gcc-9 CXX=g++-9 -cmake .. -``` -The `CC` variable specifies the compiler for C (short for C Compiler), and `CXX` variable instructs which C++ compiler is to be used for building. - -For a faster build you can resort to the `debug` build type - a build with no optimizations. For that supply the following parameter `-D CMAKE_BUILD_TYPE=Debug`: -``` -cmake -D CMAKE_BUILD_TYPE=Debug .. -``` -You can change the type of build by running this command in the `build` directory. - -Run ninja to build: -``` -ninja clickhouse-server clickhouse-client -``` -Only the required binaries are going to be built in this example. - -If you require to build all the binaries (utilities and tests), you should run ninja with no parameters: -``` -ninja -``` - -Full build requires about 30GB of free disk space or 15GB to build the main binaries. - -When large amount of RAM is available on build machine you should limit the number of build tasks run in parallel with `-j` param: -``` -ninja -j 1 clickhouse-server clickhouse-client -``` -On machines with 4GB of RAM it is recommended to specify 1, for 8GB of RAM `-j 2` is recommended. - -If you get the message: `ninja: error: loading 'build.ninja': No such file or directory`, it means that generating a build configuration has failed and you need to inspect the message above. - -Upon successful start of the building process you'll see the build progress - the number of processed tasks and the total number of tasks. - -While building messages about protobuf files in libhdfs2 library like `libprotobuf WARNING` may show up. They affect nothing and are safe to be ignored. - -Upon successful build you get an executable file `ClickHouse//dbms/programs/clickhouse`: -``` -ls -l dbms/programs/clickhouse -``` - - -# Running the built executable of ClickHouse - -To run the server under the current user you need to navigate to `ClickHouse/dbms/programs/server/` (located outside of `build`) and run: - -``` -../../../build/dbms/programs/clickhouse server -``` - -In this case ClickHouse will use config files located in the current directory. You can run `clickhouse server` from any directory specifying the path to a config file as a command line parameter `--config-file`. - -To connect to ClickHouse with clickhouse-client in another terminal navigate to `ClickHouse/build/dbms/programs/` and run `clickhouse client`. - -If you get `Connection refused` message on Mac OS X or FreeBSD, try specifying host address 127.0.0.1: -``` -clickhouse client --host 127.0.0.1 -``` - -You can replace production version of ClickHouse binary installed in your system with your custom built ClickHouse binary. To do that install ClickHouse on your machine following the instructions from the official website. Next, run the following: -``` -sudo service clickhouse-server stop -sudo cp ClickHouse/build/dbms/programs/clickhouse /usr/bin/ -sudo service clickhouse-server start -``` - -Note that `clickhouse-client`, `clickhouse-server` and others are symlinks to the commonly shared `clickhouse` binary. - -You can also run your custom built ClickHouse binary with the config file from the ClickHouse package installed on your system: -``` -sudo service clickhouse-server stop -sudo -u clickhouse ClickHouse/build/dbms/programs/clickhouse server --config-file /etc/clickhouse-server/config.xml -``` - - -# IDE (Integrated Development Environment) - -If you do not know which IDE to use, we recommend that you use CLion. CLion is a commercial software, but it offers 30 day free trial period. It is also free of charge for students. CLion can be used both on Linux and on Mac OS X. - -KDevelop and QTCreator are another great alternatives of an IDE for developing ClickHouse. KDevelop comes in as a very handy IDE although unstable. If KDevelop crashes after a while upon opening project, you should click "Stop All" button as soon as it has opened the list of project's files. After doing so KDevelop should be fine to work with. - -As simple code editors you can use Sublime Text or Visual Studio Code, or Kate (all of which are available on Linux). - -Just in case, it is worth mentioning that CLion creates by itself its own `build` path, it also selects by itself `debug` for build type, for configuration it uses a version of CMake that is defined in CLion and not the one installed by you, and finally CLion will use `make` to run build tasks instead of `ninja`. This is a normal behaviour, just keep that in mind to avoid confusion. - - -# Writing Code - -The description of ClickHouse architecture can be found here: https://clickhouse.yandex/docs/en/development/architecture/ - -The Code Style Guide: https://clickhouse.yandex/docs/en/development/style/ - -Writing tests: https://clickhouse.yandex/docs/en/development/tests/ - -List of tasks: https://github.com/yandex/ClickHouse/blob/master/dbms/tests/instructions/easy_tasks_sorted_en.md - - -# Test Data - -Developing ClickHouse often requires loading realistic datasets. It is particularly important for performance testing. We have a specially prepared set of anonymized data from Yandex.Metrica. It requires additionally some 3GB of free disk space. Note that this data is not required to accomplish most of development tasks. - -``` -sudo apt install wget xz-utils - -wget https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_v1.tsv.xz -wget https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz - -xz -v -d hits_v1.tsv.xz -xz -v -d visits_v1.tsv.xz - -clickhouse-client - -CREATE TABLE test.hits ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime); - -CREATE TABLE test.visits ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), `Goals.ID` Array(UInt32), `Goals.Serial` Array(UInt32), `Goals.EventTime` Array(DateTime), `Goals.Price` Array(Int64), `Goals.OrderID` Array(String), `Goals.CurrencyID` Array(UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, `TraficSource.ID` Array(Int8), `TraficSource.SearchEngineID` Array(UInt16), `TraficSource.AdvEngineID` Array(UInt8), `TraficSource.PlaceID` Array(UInt16), `TraficSource.SocialSourceNetworkID` Array(UInt8), `TraficSource.Domain` Array(String), `TraficSource.SearchPhrase` Array(String), `TraficSource.SocialSourcePage` Array(String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `Market.Type` Array(UInt8), `Market.GoalID` Array(UInt32), `Market.OrderID` Array(String), `Market.OrderPrice` Array(Int64), `Market.PP` Array(UInt32), `Market.DirectPlaceID` Array(UInt32), `Market.DirectOrderID` Array(UInt32), `Market.DirectBannerID` Array(UInt32), `Market.GoodID` Array(String), `Market.GoodName` Array(String), `Market.GoodQuantity` Array(Int32), `Market.GoodPrice` Array(Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID); - -clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.hits FORMAT TSV" < hits_v1.tsv -clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.visits FORMAT TSV" < visits_v1.tsv -``` - - - -# Creating Pull Request - -Navigate to your fork repository in GitHub's UI. If you have been developing in a branch, you need to select that branch. There will be a "Pull request" button located on the screen. In essence this means "create a request for accepting my changes into the main repository". - -A pull request can be created even if the work is not completed yet. In this case please put the word "WIP" (work in progress) at the beginning of the title, it can be changed later. This is useful for cooperative reviewing and discussion of changes as well as for running all of the available tests. It is important that you provide a brief description of your changes, it will later be used for generating realease changelogs. - -Testing will commence as soon as Yandex employees label your PR with a tag "can be tested". The results of some first checks (e.g. code style) will come in within several minutes. Build check results will arrive within a half an hour. And the main set of tests will report itself within an hour. - -The system will prepare ClickHouse binary builds for your pull request individually. To retrieve these builds click the "Details" link next to "ClickHouse build check" entry in the list of checks. There you will find direct links to the built .deb packages of ClickHouse which you can deploy even on your production servers (if you have no fear). - -Most probably some of the builds will fail at first times. This is due to the fact that we check builds both with gcc as well as with clang, with almost all of existing warnings (always with the `-Werror` flag) enabled for clang. On that same page you can find all of the build logs so that you do not have to build ClickHouse in all of the possible ways. diff --git a/dbms/tests/instructions/developer_instruction_en.md b/dbms/tests/instructions/developer_instruction_en.md new file mode 120000 index 00000000000..81ffff89507 --- /dev/null +++ b/dbms/tests/instructions/developer_instruction_en.md @@ -0,0 +1 @@ +../../../docs/en/development/developer_instruction.md \ No newline at end of file diff --git a/dbms/tests/instructions/developer_instruction_ru.md b/dbms/tests/instructions/developer_instruction_ru.md deleted file mode 100644 index ad03c192d52..00000000000 --- a/dbms/tests/instructions/developer_instruction_ru.md +++ /dev/null @@ -1,268 +0,0 @@ -Сборка ClickHouse поддерживается на Linux, FreeBSD, Mac OS X. - - -# Если вы используете Windows - -Если вы используете Windows, вам потребуется создать виртуальную машину с Ubuntu. Для работы с виртуальной машиной, установите VirtualBox. Скачать Ubuntu можно на сайте: https://www.ubuntu.com/#download Создайте виртуальную машину из полученного образа. Выделите для неё не менее 4 GB оперативной памяти. Для запуска терминала в Ubuntu, найдите в меню программу со словом terminal (gnome-terminal, konsole или что-то в этом роде) или нажмите Ctrl+Alt+T. - - -# Создание репозитория на GitHub - -Для работы с репозиторием ClickHouse, вам потребуется аккаунт на GitHub. Наверное, он у вас уже есть. - -Если аккаунта нет - зарегистрируйтесь на https://github.com/. Создайте ssh ключи, если их нет, и загрузите публичные ключи на GitHub. Это потребуется для отправки изменений. Для работы с GitHub можно использовать такие же ssh ключи, как и для работы с другими ssh серверами - скорее всего, они уже у вас есть. - -Создайте fork репозитория ClickHouse. Для этого, на странице https://github.com/ClickHouse/ClickHouse нажмите на кнопку "fork" в правом верхнем углу. Вы получите полную копию репозитория ClickHouse на своём аккаунте, которая называется "форк". Процесс разработки состоит в том, чтобы внести нужные изменения в свой форк репозитория, а затем создать "pull request" для принятия изменений в основной репозиторий. - -Для работы с git репозиториями, установите `git`. - -В Ubuntu выполните в терминале: -``` -sudo apt update -sudo apt install git -``` - -Краткое руководство по использованию Git: https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf - -Подробное руководство по использованию Git: https://git-scm.com/book/ru/v2 - - -# Клонирование репозитория на рабочую машину - -Затем вам потребуется загрузить исходники для работы на свой компьютер. Это называется "клонирование репозитория", потому что создаёт на вашем компьютере локальную копию репозитория, с которой вы будете работать. - -Выполните в терминале: -``` -git clone --recursive git@github.com:yandex/ClickHouse.git -cd ClickHouse -``` -Замените *yandex* на имя вашего аккаунта на GitHub. - -Эта команда создаст директорию ClickHouse, содержащую рабочую копию проекта. - -Необходимо, чтобы путь к рабочей копии не содержал пробелы в именах директорий. Это может привести к проблемам в работе системы сборки. - -Обратите внимание, что репозиторий ClickHouse использует submodules. Так называются ссылки на дополнительные репозитории (например, внешние библиотеки, от которых зависит проект). Это значит, что при клонировании репозитория, следует указывать ключ `--recursive`, как в примере выше. Если репозиторий был клонирован без submodules, то для их скачивания, необходимо выполнить: -``` -git submodule init -git submodule update -``` -Проверить наличие submodules можно с помощью команды `git submodule status`. - -Если вы получили сообщение об ошибке: -``` -Permission denied (publickey). -fatal: Could not read from remote repository. - -Please make sure you have the correct access rights -and the repository exists. -``` -Как правило это означает, что отсутствуют ssh ключи для соединения с GitHub. Ключи расположены в директории `~/.ssh`. В интерфейсе GitHub, в настройках, необходимо загрузить публичные ключи, чтобы он их понимал. - -Вы также можете клонировать репозиторий по протоколу https: -``` -git clone https://github.com/ClickHouse/ClickHouse.git -``` -Этот вариант не подходит для отправки изменений на сервер. Вы можете временно его использовать, а затем добавить ssh ключи и заменить адрес репозитория с помощью команды `git remote`. - -Вы можете также добавить для своего локального репозитория адрес оригинального репозитория Яндекса, чтобы притягивать оттуда обновления: -``` -git remote add upstream git@github.com:yandex/ClickHouse.git -``` -После этого, вы сможете добавлять в свой репозиторий обновления из репозитория Яндекса с помощью команды `git pull upstream master`. - - -# Система сборки - -ClickHouse использует систему сборки CMake и Ninja. - -CMake - генератор задач сборки. -Ninja - система запуска сборочных задач. - -Для установки на Ubuntu или Debian, Mint, выполните `sudo apt install cmake ninja-build`. - -Для установки на CentOS, RedHat, выполните `sudo yum install cmake ninja-build`. - -Если у вас Arch или Gentoo, то вы сами знаете, как установить CMake. - -Для установки CMake и Ninja на Mac OS X, сначала установите Homebrew, а затем, с помощью него, установите всё остальное. -``` -/usr/bin/ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/install)" -brew install cmake ninja -``` - -Проверьте версию CMake: `cmake --version`. Если версия меньше 3.3, то установите новую версию с сайта https://cmake.org/download/ - - -# Необязательные внешние библиотеки - -ClickHouse использует для сборки некоторое количество внешних библиотек. Большинство из них не требуется отдельно устанавливать, так как они собираются вместе с ClickHouse, из исходников, которые расположены в submodules. Посмотреть набор этих библиотек можно в директории contrib. - -Пара библиотек не собирается из исходников, а используется из системы: ICU и Readline, и их рекомендуется установить. - -Ubuntu: `sudo apt install libicu-dev libreadline-dev` - -Mac OS X: `brew install icu4c readline` - -Впрочем, эти библиотеки не обязательны для работы и ClickHouse может быть собран без них. ICU используется для поддержки `COLLATE` в `ORDER BY` (например, для сортировки с учётом турецкого алфавита). Readline используется для более удобного набора команд в интерактивном режиме в clickhouse-client. - - -# Компилятор C++ - -В качестве компилятора C++ поддерживается GCC начиная с версии 9 или Clang начиная с версии 8. - -Официальные сборки от Яндекса, на данный момент, используют GCC, так как он генерирует слегка более производительный машинный код (разница в среднем до нескольких процентов по нашим бенчмаркам). Clang обычно более удобен для разработки. Впрочем, наша среда continuous integration проверяет около десятка вариантов сборки. - -Для установки GCC под Ubuntu, выполните: `sudo apt install gcc g++`. - -Проверьте версию gcc: `gcc --version`. Если версия меньше 9, то следуйте инструкции: https://clickhouse.yandex/docs/en/development/build/#install-gcc-9 - -Для установки GCC под Mac OS X, выполните `brew install gcc`. - -Если вы решили использовать Clang, вы также можете установить `libc++` и `lld`, если вы знаете, что это такое. При желании, установите `ccache`. - - -# Процесс сборки - -Теперь вы готовы к сборке ClickHouse. Для размещения собранных файлов, рекомендуется создать отдельную директорию build внутри директории ClickHouse: -``` -mkdir build -cd build -``` -Вы можете иметь несколько разных директорий (build_release, build_debug) для разных вариантов сборки. - -Находясь в директории build, выполните конфигурацию сборки с помощью CMake. -Перед первым запуском необходимо выставить переменные окружения, отвечающие за выбор компилятора (в данном примере это - gcc версии 9). -``` -export CC=gcc-9 CXX=g++-9 -cmake .. -``` -Переменная CC отвечает за компилятор C (сокращение от слов C Compiler), переменная CXX отвечает за выбор компилятора C++ (символ X - это как плюс, но положенный набок, ради того, чтобы превратить его в букву). - -Для более быстрой сборки, можно использовать debug вариант - сборку без оптимизаций. Для этого, укажите параметр `-D CMAKE_BUILD_TYPE=Debug`: -``` -cmake -D CMAKE_BUILD_TYPE=Debug .. -``` -Вы можете изменить вариант сборки, выполнив эту команду в директории build. - -Запустите ninja для сборки: -``` -ninja clickhouse-server clickhouse-client -``` -В этом примере собираются только нужные в первую очередь программы. - -Если вы хотите собрать все программы (утилиты и тесты), то запустите ninja без параметров: -``` -ninja -``` - -Для полной сборки требуется около 30 GB свободного места на диске или 15 GB для сборки только основных программ. - -При наличии небольшого количества оперативной памяти на компьютере, следует ограничить количество параллельных задач с помощью параметра `-j`: -``` -ninja -j 1 clickhouse-server clickhouse-client -``` -На машинах с 4 GB памяти, рекомендуется указывать значение 1, а если памяти до 8 GB, укажите значение 2. - -Если вы получили сообщение `ninja: error: loading 'build.ninja': No such file or directory`, значит конфигурация сборки прошла с ошибкой и вам необходимо посмотреть на сообщение об ошибке выше. - -В случае успешного запуска, вы увидите прогресс сборки - количество обработанных задач и общее количество задач. - -В процессе сборки могут появится сообщения `libprotobuf WARNING` про protobuf файлы в библиотеке libhdfs2. Это не имеет значения. - -При успешной сборке, вы получите готовый исполняемый файл `ClickHouse/build/dbms/programs/clickhouse`: -``` -ls -l dbms/programs/clickhouse -``` - - -# Запуск собранной версии ClickHouse - -Для запуска сервера из под текущего пользователя, с выводом логов в терминал и с использованием примеров конфигурационных файлов, расположенных в исходниках, перейдите в директорию `ClickHouse/dbms/programs/server/` (эта директория находится не в директории build) и выполните: - -``` -../../../build/dbms/programs/clickhouse server -``` - -В этом случае, ClickHouse будет использовать конфигурационные файлы, расположенные в текущей директории. Вы можете запустить `clickhouse server` из любой директории, передав ему путь к конфигурационному файлу в аргументе командной строки `--config-file`. - -Для подключения к ClickHouse с помощью clickhouse-client, в соседнем терминале, зайдите в директорию `ClickHouse/build/dbms/programs/` и выполните `clickhouse client`. - -Если вы получили сообщение `Connection refused` на Mac OS X или FreeBSD, то укажите для клиента 127.0.0.1 в качестве имени хоста: -``` -clickhouse client --host 127.0.0.1 -``` - -Вы можете заменить собранным вами ClickHouse продакшен версию, установленную в системе. Для этого, установите ClickHouse на свою машину по инструкции с официального сайта. Затем выполните: -``` -sudo service clickhouse-server stop -sudo cp ClickHouse/build/dbms/programs/clickhouse /usr/bin/ -sudo service clickhouse-server start -``` - -Обратите внимание, что `clickhouse-client`, `clickhouse-server` и другие, являеются симлинками на общий бинарник `clickhouse`. - -Также вы можете запустить собранный вами ClickHouse с конфигурационным файлом системного ClickHouse: -``` -sudo service clickhouse-server stop -sudo -u clickhouse ClickHouse/build/dbms/programs/clickhouse server --config-file /etc/clickhouse-server/config.xml -``` - - -# Среда разработки - -Если вы не знаете, какую среду разработки использовать, то рекомендуется использовать CLion. CLion является платным ПО, но его можно использовать бесплатно в течение пробного периода. Также он бесплатен для учащихся. CLion можно использовать как под Linux, так и под Mac OS X. - -Также в качестве среды разработки, вы можете использовать KDevelop или QTCreator. KDevelop - очень удобная, но нестабильная среда разработки. Если KDevelop вылетает через небольшое время после открытия проекта, вам следует нажать на кнопку "Stop All" как только он открыл список файлов проекта. После этого, KDevelop можно будет использовать. - -В качестве простых редакторов кода можно использовать Sublime Text или Visual Studio Code или Kate (все варианты доступны под Linux). - -На всякий случай заметим, что CLion самостоятельно создаёт свою build директорию, самостоятельно выбирает тип сборки debug по-умолчанию, для конфигурации использует встроенную в CLion версию CMake вместо установленного вами, а для запуска задач использует make вместо ninja. Это нормально, просто имейте это ввиду, чтобы не возникало путаницы. - - -# Написание кода - -Описание архитектуры ClickHouse: https://clickhouse.yandex/docs/ru/development/architecture/ - -Стиль кода: https://clickhouse.yandex/docs/ru/development/style/ - -Разработка тестов: https://clickhouse.yandex/docs/ru/development/tests/ - -Список задач: https://github.com/ClickHouse/ClickHouse/blob/master/dbms/tests/instructions/easy_tasks_sorted_ru.md - - -# Тестовые данные - -Разработка ClickHouse часто требует загрузки реалистичных наборов данных. Особенно это важно для тестирования производительности. Специально для вас мы подготовили набор данных, представляющий собой анонимизированные данные Яндекс.Метрики. Загрузка этих данных потребует ещё 3 GB места на диске. Для выполнения большинства задач разработки, загружать эти данные не обязательно. - -``` -sudo apt install wget xz-utils - -wget https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_v1.tsv.xz -wget https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz - -xz -v -d hits_v1.tsv.xz -xz -v -d visits_v1.tsv.xz - -clickhouse-client - -CREATE TABLE test.hits ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime); - -CREATE TABLE test.visits ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), `Goals.ID` Array(UInt32), `Goals.Serial` Array(UInt32), `Goals.EventTime` Array(DateTime), `Goals.Price` Array(Int64), `Goals.OrderID` Array(String), `Goals.CurrencyID` Array(UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, `TraficSource.ID` Array(Int8), `TraficSource.SearchEngineID` Array(UInt16), `TraficSource.AdvEngineID` Array(UInt8), `TraficSource.PlaceID` Array(UInt16), `TraficSource.SocialSourceNetworkID` Array(UInt8), `TraficSource.Domain` Array(String), `TraficSource.SearchPhrase` Array(String), `TraficSource.SocialSourcePage` Array(String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `Market.Type` Array(UInt8), `Market.GoalID` Array(UInt32), `Market.OrderID` Array(String), `Market.OrderPrice` Array(Int64), `Market.PP` Array(UInt32), `Market.DirectPlaceID` Array(UInt32), `Market.DirectOrderID` Array(UInt32), `Market.DirectBannerID` Array(UInt32), `Market.GoodID` Array(String), `Market.GoodName` Array(String), `Market.GoodQuantity` Array(Int32), `Market.GoodPrice` Array(Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID); - -clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.hits FORMAT TSV" < hits_v1.tsv -clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.visits FORMAT TSV" < visits_v1.tsv -``` - - -# Создание pull request - -Откройте свой форк репозитория в интерфейсе GitHub. Если вы вели разработку в бранче, выберите этот бранч. На странице будет доступна кнопка "Pull request". По сути, это означает "создать заявку на принятие моих изменений в основной репозиторий". - -Pull request можно создать, даже если работа над задачей ещё не завершена. В этом случае, добавьте в его название слово "WIP" (work in progress). Название можно будет изменить позже. Это полезно для совместного просмотра и обсуждения изменений, а также для запуска всех имеющихся тестов. Введите краткое описание изменений - впоследствии, оно будет использовано для релизных changelog. - -Тесты будут запущены, как только сотрудники Яндекса поставят для pull request тег "Can be tested". Результаты первых проверок (стиль кода) появятся уже через несколько минут. Результаты сборки появятся примерно через пол часа. Результаты основного набора тестов будут доступны в пределах часа. - -Система подготовит сборки ClickHouse специально для вашего pull request. Для их получения, нажмите на ссылку "Details" у проверки "Clickhouse build check". Там вы сможете найти прямые ссылки на собранные .deb пакеты ClickHouse, которые, при желании, вы даже сможете установить на свои продакшен серверы (если не страшно). - -Вероятнее всего, часть сборок не будет успешной с первого раза. Ведь мы проверяем сборку кода и gcc и clang, а при сборке с помощью clang включаются почти все существующие в природе warnings (всегда с флагом `-Werror`). На той же странице, вы сможете найти логи сборки - вам не обязательно самому собирать ClickHouse всеми возможными способами. diff --git a/dbms/tests/instructions/developer_instruction_ru.md b/dbms/tests/instructions/developer_instruction_ru.md new file mode 120000 index 00000000000..9f912ebfec9 --- /dev/null +++ b/dbms/tests/instructions/developer_instruction_ru.md @@ -0,0 +1 @@ +../../../docs/ru/development/developer_instruction.md \ No newline at end of file diff --git a/docs/en/development/developer_instruction.md b/docs/en/development/developer_instruction.md new file mode 100644 index 00000000000..463f186fb3b --- /dev/null +++ b/docs/en/development/developer_instruction.md @@ -0,0 +1,268 @@ +Building of ClickHouse is supported on Linux, FreeBSD and Mac OS X. + +# If you use Windows + +If you use Windows, you need to create a virtual machine with Ubuntu. To start working with a virtual machine please install VirtualBox. You can download Ubuntu from the website: https://www.ubuntu.com/#download. Please create a virtual machine from the downloaded image (you should reserve at least 4GB of RAM for it). To run a command line terminal in Ubuntu, please locate a program containing the word "terminal" in its name (gnome-terminal, konsole etc.) or just press Ctrl+Alt+T. + + +# Creating a repository on GitHub + +To start working with ClickHouse repository you will need a GitHub account. + +You probably already have one, but if you don't, please register at https://github.com. In case you do not have SSH keys, you should generate them and then upload them on GitHub. It is required for sending over your patches. It is also possible to use the same SSH keys that you use with any other SSH servers - probably you already have those. + +Create a fork of ClickHouse repository. To do that please click on the "fork" button in the upper right corner at https://github.com/ClickHouse/ClickHouse. It will fork your own copy of ClickHouse/ClickHouse to your account. + +Development process consists of first committing the intended changes into your fork of ClickHouse and then creating a "pull request" for these changes to be accepted into the main repository (ClickHouse/ClickHouse). + +To work with git repositories, please install `git`. + +To do that in Ubuntu you would run in the command line terminal: +``` +sudo apt update +sudo apt install git +``` + +A brief manual on using Git can be found here: https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf. +For a detailed manual on Git see: https://git-scm.com/book/ru/v2. + + +# Cloning a repository to your development machine + +Next, you need to download the source files onto your working machine. This is called "to clone a repository" because it creates a local copy of the repository on your working machine. + +In the command line terminal run: +``` +git clone --recursive git@guthub.com:your_github_username/ClickHouse.git +cd ClickHouse +``` +Note: please, substitute *your_github_username* with what is appropriate! + +This command will create a directory `ClickHouse` containing the working copy of the project. + +It is important that the path to the working directory contains no whitespaces as it may lead to problems with running the build system. + +Please note that ClickHouse repository uses `submodules`. That is what the references to additional repositories are called (i.e. external libraries on which the project depends). It means that when cloning the repository you need to specify the `--recursive` flag as in the example above. If the repository has been cloned without submodules, to download them you need to run the following: +``` +git submodule init +git submodule update +``` +You can check status with command: `git submodule status`. + +If you get the following error message: +``` +Permission denied (publickey). +fatal: Could not read from remote repository. + +Please make sure you have the correct access rights +and the repository exists. +``` +It generally means that the SSH keys for connecting to GitHub are missing. These keys are normally located in `~/.ssh`. For SSH keys to be accepted you need to upload them in the settings section of GitHub UI. + +You can also clone the repository via https protocol: +``` +git clone https://github.com/ClickHouse/ClickHouse.git +``` +This however will not let you send your changes to the server. You can still use it temporarily and add the SSH keys later replacing the remote address of the repository with `git remote` command. + +You can also add original ClickHouse repo's address to your local repository to pull updates from there: +``` +git remote add upstream git@github.com:ClickHouse/ClickHouse.git +``` +After successfully running this command you will be able to pull updates from the main ClickHouse repo by running `git pull upstream master`. + + +# Build System + +ClickHouse uses CMake and Ninja for building. + +CMake - a meta-build system that can generate Ninja files (build tasks). +Ninja - a smaller build system with focus on speed used to execute those cmake generated tasks. + +To install on Ubuntu, Debian or Mint run `sudo apt install cmake ninja-build`. + +On CentOS, RedHat run `sudo yum install cmake ninja-build`. + +If you use Arch or Gentoo, you probably know it yourself how to install CMake. + +For installing CMake and Ninja on Mac OS X first install Homebrew and then install everything else via brew: +``` +/usr/bin/ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/install)" +brew install cmake ninja +``` + +Next, check the version of CMake: `cmake --version`. If it is below 3.3, you should install a newer version from the website: https://cmake.org/download/. + + +# Optional External Libraries + +ClickHouse uses several external libraries for building. Most of them do not need to be installed separately as they are built together with ClickHouse from the sources located in the submodules. You can check the list in `contrib`. + +There is a couple of libraries that are not built from sources but are supplied by the system: ICU and Readline, and thus are recommended to be installed. + +Ubuntu: `sudo apt install libicu-dev libreadline-dev` + +Mac OS X: `brew install icu4c readline` + +However, these libraries are optional and ClickHouse can well be built without them. ICU is used for support of `COLLATE` in `ORDER BY` (i.e. for sorting in turkish alphabet). Readline is used for more convenient command input in clickhouse-client. + + +# C++ Compiler + +Compilers GCC starting from version 9 and Clang version 8 or above are supported for building ClickHouse. + +Official Yandex builds currently use GCC because it generates machine code of slightly better performance (yielding a difference of up to several percent according to our benchmarks). And Clang is more convenient for development usually. Though, our continuous integration (CI) platform runs checks for about a dozen of build combinations. + +To install GCC on Ubuntu run: `sudo apt install gcc g++` + +Check the version of gcc: `gcc --version`. If it is below 9, then follow the instruction here: https://clickhouse.yandex/docs/en/development/build/#install-gcc-9. + +To install GCC on Mac OS X run: `brew install gcc`. + +If you decide to use Clang, you can also install `libc++` and `lld`, if you know what it is. Using `ccache` is also recommended. + + +# The Building process + +Now that you are ready to build ClickHouse we recommend you to create a separate directory `build` inside `ClickHouse` that will contain all of the build artefacts: +``` +mkdir build +cd build +``` +You can have several different directories (build_release, build_debug, etc.) for different types of build. + +While inside the `build` directory, configure your build by running CMake. Before the first run you need to define environment variables that specify compiler (version 9 gcc compiler in this example). +``` +export CC=gcc-9 CXX=g++-9 +cmake .. +``` +The `CC` variable specifies the compiler for C (short for C Compiler), and `CXX` variable instructs which C++ compiler is to be used for building. + +For a faster build you can resort to the `debug` build type - a build with no optimizations. For that supply the following parameter `-D CMAKE_BUILD_TYPE=Debug`: +``` +cmake -D CMAKE_BUILD_TYPE=Debug .. +``` +You can change the type of build by running this command in the `build` directory. + +Run ninja to build: +``` +ninja clickhouse-server clickhouse-client +``` +Only the required binaries are going to be built in this example. + +If you require to build all the binaries (utilities and tests), you should run ninja with no parameters: +``` +ninja +``` + +Full build requires about 30GB of free disk space or 15GB to build the main binaries. + +When large amount of RAM is available on build machine you should limit the number of build tasks run in parallel with `-j` param: +``` +ninja -j 1 clickhouse-server clickhouse-client +``` +On machines with 4GB of RAM it is recommended to specify 1, for 8GB of RAM `-j 2` is recommended. + +If you get the message: `ninja: error: loading 'build.ninja': No such file or directory`, it means that generating a build configuration has failed and you need to inspect the message above. + +Upon successful start of the building process you'll see the build progress - the number of processed tasks and the total number of tasks. + +While building messages about protobuf files in libhdfs2 library like `libprotobuf WARNING` may show up. They affect nothing and are safe to be ignored. + +Upon successful build you get an executable file `ClickHouse//dbms/programs/clickhouse`: +``` +ls -l dbms/programs/clickhouse +``` + + +# Running the built executable of ClickHouse + +To run the server under the current user you need to navigate to `ClickHouse/dbms/programs/server/` (located outside of `build`) and run: + +``` +../../../build/dbms/programs/clickhouse server +``` + +In this case ClickHouse will use config files located in the current directory. You can run `clickhouse server` from any directory specifying the path to a config file as a command line parameter `--config-file`. + +To connect to ClickHouse with clickhouse-client in another terminal navigate to `ClickHouse/build/dbms/programs/` and run `clickhouse client`. + +If you get `Connection refused` message on Mac OS X or FreeBSD, try specifying host address 127.0.0.1: +``` +clickhouse client --host 127.0.0.1 +``` + +You can replace production version of ClickHouse binary installed in your system with your custom built ClickHouse binary. To do that install ClickHouse on your machine following the instructions from the official website. Next, run the following: +``` +sudo service clickhouse-server stop +sudo cp ClickHouse/build/dbms/programs/clickhouse /usr/bin/ +sudo service clickhouse-server start +``` + +Note that `clickhouse-client`, `clickhouse-server` and others are symlinks to the commonly shared `clickhouse` binary. + +You can also run your custom built ClickHouse binary with the config file from the ClickHouse package installed on your system: +``` +sudo service clickhouse-server stop +sudo -u clickhouse ClickHouse/build/dbms/programs/clickhouse server --config-file /etc/clickhouse-server/config.xml +``` + + +# IDE (Integrated Development Environment) + +If you do not know which IDE to use, we recommend that you use CLion. CLion is a commercial software, but it offers 30 day free trial period. It is also free of charge for students. CLion can be used both on Linux and on Mac OS X. + +KDevelop and QTCreator are another great alternatives of an IDE for developing ClickHouse. KDevelop comes in as a very handy IDE although unstable. If KDevelop crashes after a while upon opening project, you should click "Stop All" button as soon as it has opened the list of project's files. After doing so KDevelop should be fine to work with. + +As simple code editors you can use Sublime Text or Visual Studio Code, or Kate (all of which are available on Linux). + +Just in case, it is worth mentioning that CLion creates by itself its own `build` path, it also selects by itself `debug` for build type, for configuration it uses a version of CMake that is defined in CLion and not the one installed by you, and finally CLion will use `make` to run build tasks instead of `ninja`. This is a normal behaviour, just keep that in mind to avoid confusion. + + +# Writing Code + +The description of ClickHouse architecture can be found here: https://clickhouse.yandex/docs/en/development/architecture/ + +The Code Style Guide: https://clickhouse.yandex/docs/en/development/style/ + +Writing tests: https://clickhouse.yandex/docs/en/development/tests/ + +List of tasks: https://github.com/yandex/ClickHouse/blob/master/dbms/tests/instructions/easy_tasks_sorted_en.md + + +# Test Data + +Developing ClickHouse often requires loading realistic datasets. It is particularly important for performance testing. We have a specially prepared set of anonymized data from Yandex.Metrica. It requires additionally some 3GB of free disk space. Note that this data is not required to accomplish most of development tasks. + +``` +sudo apt install wget xz-utils + +wget https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_v1.tsv.xz +wget https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz + +xz -v -d hits_v1.tsv.xz +xz -v -d visits_v1.tsv.xz + +clickhouse-client + +CREATE TABLE test.hits ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime); + +CREATE TABLE test.visits ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), `Goals.ID` Array(UInt32), `Goals.Serial` Array(UInt32), `Goals.EventTime` Array(DateTime), `Goals.Price` Array(Int64), `Goals.OrderID` Array(String), `Goals.CurrencyID` Array(UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, `TraficSource.ID` Array(Int8), `TraficSource.SearchEngineID` Array(UInt16), `TraficSource.AdvEngineID` Array(UInt8), `TraficSource.PlaceID` Array(UInt16), `TraficSource.SocialSourceNetworkID` Array(UInt8), `TraficSource.Domain` Array(String), `TraficSource.SearchPhrase` Array(String), `TraficSource.SocialSourcePage` Array(String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `Market.Type` Array(UInt8), `Market.GoalID` Array(UInt32), `Market.OrderID` Array(String), `Market.OrderPrice` Array(Int64), `Market.PP` Array(UInt32), `Market.DirectPlaceID` Array(UInt32), `Market.DirectOrderID` Array(UInt32), `Market.DirectBannerID` Array(UInt32), `Market.GoodID` Array(String), `Market.GoodName` Array(String), `Market.GoodQuantity` Array(Int32), `Market.GoodPrice` Array(Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID); + +clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.hits FORMAT TSV" < hits_v1.tsv +clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.visits FORMAT TSV" < visits_v1.tsv +``` + + + +# Creating Pull Request + +Navigate to your fork repository in GitHub's UI. If you have been developing in a branch, you need to select that branch. There will be a "Pull request" button located on the screen. In essence this means "create a request for accepting my changes into the main repository". + +A pull request can be created even if the work is not completed yet. In this case please put the word "WIP" (work in progress) at the beginning of the title, it can be changed later. This is useful for cooperative reviewing and discussion of changes as well as for running all of the available tests. It is important that you provide a brief description of your changes, it will later be used for generating realease changelogs. + +Testing will commence as soon as Yandex employees label your PR with a tag "can be tested". The results of some first checks (e.g. code style) will come in within several minutes. Build check results will arrive within a half an hour. And the main set of tests will report itself within an hour. + +The system will prepare ClickHouse binary builds for your pull request individually. To retrieve these builds click the "Details" link next to "ClickHouse build check" entry in the list of checks. There you will find direct links to the built .deb packages of ClickHouse which you can deploy even on your production servers (if you have no fear). + +Most probably some of the builds will fail at first times. This is due to the fact that we check builds both with gcc as well as with clang, with almost all of existing warnings (always with the `-Werror` flag) enabled for clang. On that same page you can find all of the build logs so that you do not have to build ClickHouse in all of the possible ways. diff --git a/docs/ru/development/developer_instruction.md b/docs/ru/development/developer_instruction.md new file mode 100644 index 00000000000..ad03c192d52 --- /dev/null +++ b/docs/ru/development/developer_instruction.md @@ -0,0 +1,268 @@ +Сборка ClickHouse поддерживается на Linux, FreeBSD, Mac OS X. + + +# Если вы используете Windows + +Если вы используете Windows, вам потребуется создать виртуальную машину с Ubuntu. Для работы с виртуальной машиной, установите VirtualBox. Скачать Ubuntu можно на сайте: https://www.ubuntu.com/#download Создайте виртуальную машину из полученного образа. Выделите для неё не менее 4 GB оперативной памяти. Для запуска терминала в Ubuntu, найдите в меню программу со словом terminal (gnome-terminal, konsole или что-то в этом роде) или нажмите Ctrl+Alt+T. + + +# Создание репозитория на GitHub + +Для работы с репозиторием ClickHouse, вам потребуется аккаунт на GitHub. Наверное, он у вас уже есть. + +Если аккаунта нет - зарегистрируйтесь на https://github.com/. Создайте ssh ключи, если их нет, и загрузите публичные ключи на GitHub. Это потребуется для отправки изменений. Для работы с GitHub можно использовать такие же ssh ключи, как и для работы с другими ssh серверами - скорее всего, они уже у вас есть. + +Создайте fork репозитория ClickHouse. Для этого, на странице https://github.com/ClickHouse/ClickHouse нажмите на кнопку "fork" в правом верхнем углу. Вы получите полную копию репозитория ClickHouse на своём аккаунте, которая называется "форк". Процесс разработки состоит в том, чтобы внести нужные изменения в свой форк репозитория, а затем создать "pull request" для принятия изменений в основной репозиторий. + +Для работы с git репозиториями, установите `git`. + +В Ubuntu выполните в терминале: +``` +sudo apt update +sudo apt install git +``` + +Краткое руководство по использованию Git: https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf + +Подробное руководство по использованию Git: https://git-scm.com/book/ru/v2 + + +# Клонирование репозитория на рабочую машину + +Затем вам потребуется загрузить исходники для работы на свой компьютер. Это называется "клонирование репозитория", потому что создаёт на вашем компьютере локальную копию репозитория, с которой вы будете работать. + +Выполните в терминале: +``` +git clone --recursive git@github.com:yandex/ClickHouse.git +cd ClickHouse +``` +Замените *yandex* на имя вашего аккаунта на GitHub. + +Эта команда создаст директорию ClickHouse, содержащую рабочую копию проекта. + +Необходимо, чтобы путь к рабочей копии не содержал пробелы в именах директорий. Это может привести к проблемам в работе системы сборки. + +Обратите внимание, что репозиторий ClickHouse использует submodules. Так называются ссылки на дополнительные репозитории (например, внешние библиотеки, от которых зависит проект). Это значит, что при клонировании репозитория, следует указывать ключ `--recursive`, как в примере выше. Если репозиторий был клонирован без submodules, то для их скачивания, необходимо выполнить: +``` +git submodule init +git submodule update +``` +Проверить наличие submodules можно с помощью команды `git submodule status`. + +Если вы получили сообщение об ошибке: +``` +Permission denied (publickey). +fatal: Could not read from remote repository. + +Please make sure you have the correct access rights +and the repository exists. +``` +Как правило это означает, что отсутствуют ssh ключи для соединения с GitHub. Ключи расположены в директории `~/.ssh`. В интерфейсе GitHub, в настройках, необходимо загрузить публичные ключи, чтобы он их понимал. + +Вы также можете клонировать репозиторий по протоколу https: +``` +git clone https://github.com/ClickHouse/ClickHouse.git +``` +Этот вариант не подходит для отправки изменений на сервер. Вы можете временно его использовать, а затем добавить ssh ключи и заменить адрес репозитория с помощью команды `git remote`. + +Вы можете также добавить для своего локального репозитория адрес оригинального репозитория Яндекса, чтобы притягивать оттуда обновления: +``` +git remote add upstream git@github.com:yandex/ClickHouse.git +``` +После этого, вы сможете добавлять в свой репозиторий обновления из репозитория Яндекса с помощью команды `git pull upstream master`. + + +# Система сборки + +ClickHouse использует систему сборки CMake и Ninja. + +CMake - генератор задач сборки. +Ninja - система запуска сборочных задач. + +Для установки на Ubuntu или Debian, Mint, выполните `sudo apt install cmake ninja-build`. + +Для установки на CentOS, RedHat, выполните `sudo yum install cmake ninja-build`. + +Если у вас Arch или Gentoo, то вы сами знаете, как установить CMake. + +Для установки CMake и Ninja на Mac OS X, сначала установите Homebrew, а затем, с помощью него, установите всё остальное. +``` +/usr/bin/ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/install)" +brew install cmake ninja +``` + +Проверьте версию CMake: `cmake --version`. Если версия меньше 3.3, то установите новую версию с сайта https://cmake.org/download/ + + +# Необязательные внешние библиотеки + +ClickHouse использует для сборки некоторое количество внешних библиотек. Большинство из них не требуется отдельно устанавливать, так как они собираются вместе с ClickHouse, из исходников, которые расположены в submodules. Посмотреть набор этих библиотек можно в директории contrib. + +Пара библиотек не собирается из исходников, а используется из системы: ICU и Readline, и их рекомендуется установить. + +Ubuntu: `sudo apt install libicu-dev libreadline-dev` + +Mac OS X: `brew install icu4c readline` + +Впрочем, эти библиотеки не обязательны для работы и ClickHouse может быть собран без них. ICU используется для поддержки `COLLATE` в `ORDER BY` (например, для сортировки с учётом турецкого алфавита). Readline используется для более удобного набора команд в интерактивном режиме в clickhouse-client. + + +# Компилятор C++ + +В качестве компилятора C++ поддерживается GCC начиная с версии 9 или Clang начиная с версии 8. + +Официальные сборки от Яндекса, на данный момент, используют GCC, так как он генерирует слегка более производительный машинный код (разница в среднем до нескольких процентов по нашим бенчмаркам). Clang обычно более удобен для разработки. Впрочем, наша среда continuous integration проверяет около десятка вариантов сборки. + +Для установки GCC под Ubuntu, выполните: `sudo apt install gcc g++`. + +Проверьте версию gcc: `gcc --version`. Если версия меньше 9, то следуйте инструкции: https://clickhouse.yandex/docs/en/development/build/#install-gcc-9 + +Для установки GCC под Mac OS X, выполните `brew install gcc`. + +Если вы решили использовать Clang, вы также можете установить `libc++` и `lld`, если вы знаете, что это такое. При желании, установите `ccache`. + + +# Процесс сборки + +Теперь вы готовы к сборке ClickHouse. Для размещения собранных файлов, рекомендуется создать отдельную директорию build внутри директории ClickHouse: +``` +mkdir build +cd build +``` +Вы можете иметь несколько разных директорий (build_release, build_debug) для разных вариантов сборки. + +Находясь в директории build, выполните конфигурацию сборки с помощью CMake. +Перед первым запуском необходимо выставить переменные окружения, отвечающие за выбор компилятора (в данном примере это - gcc версии 9). +``` +export CC=gcc-9 CXX=g++-9 +cmake .. +``` +Переменная CC отвечает за компилятор C (сокращение от слов C Compiler), переменная CXX отвечает за выбор компилятора C++ (символ X - это как плюс, но положенный набок, ради того, чтобы превратить его в букву). + +Для более быстрой сборки, можно использовать debug вариант - сборку без оптимизаций. Для этого, укажите параметр `-D CMAKE_BUILD_TYPE=Debug`: +``` +cmake -D CMAKE_BUILD_TYPE=Debug .. +``` +Вы можете изменить вариант сборки, выполнив эту команду в директории build. + +Запустите ninja для сборки: +``` +ninja clickhouse-server clickhouse-client +``` +В этом примере собираются только нужные в первую очередь программы. + +Если вы хотите собрать все программы (утилиты и тесты), то запустите ninja без параметров: +``` +ninja +``` + +Для полной сборки требуется около 30 GB свободного места на диске или 15 GB для сборки только основных программ. + +При наличии небольшого количества оперативной памяти на компьютере, следует ограничить количество параллельных задач с помощью параметра `-j`: +``` +ninja -j 1 clickhouse-server clickhouse-client +``` +На машинах с 4 GB памяти, рекомендуется указывать значение 1, а если памяти до 8 GB, укажите значение 2. + +Если вы получили сообщение `ninja: error: loading 'build.ninja': No such file or directory`, значит конфигурация сборки прошла с ошибкой и вам необходимо посмотреть на сообщение об ошибке выше. + +В случае успешного запуска, вы увидите прогресс сборки - количество обработанных задач и общее количество задач. + +В процессе сборки могут появится сообщения `libprotobuf WARNING` про protobuf файлы в библиотеке libhdfs2. Это не имеет значения. + +При успешной сборке, вы получите готовый исполняемый файл `ClickHouse/build/dbms/programs/clickhouse`: +``` +ls -l dbms/programs/clickhouse +``` + + +# Запуск собранной версии ClickHouse + +Для запуска сервера из под текущего пользователя, с выводом логов в терминал и с использованием примеров конфигурационных файлов, расположенных в исходниках, перейдите в директорию `ClickHouse/dbms/programs/server/` (эта директория находится не в директории build) и выполните: + +``` +../../../build/dbms/programs/clickhouse server +``` + +В этом случае, ClickHouse будет использовать конфигурационные файлы, расположенные в текущей директории. Вы можете запустить `clickhouse server` из любой директории, передав ему путь к конфигурационному файлу в аргументе командной строки `--config-file`. + +Для подключения к ClickHouse с помощью clickhouse-client, в соседнем терминале, зайдите в директорию `ClickHouse/build/dbms/programs/` и выполните `clickhouse client`. + +Если вы получили сообщение `Connection refused` на Mac OS X или FreeBSD, то укажите для клиента 127.0.0.1 в качестве имени хоста: +``` +clickhouse client --host 127.0.0.1 +``` + +Вы можете заменить собранным вами ClickHouse продакшен версию, установленную в системе. Для этого, установите ClickHouse на свою машину по инструкции с официального сайта. Затем выполните: +``` +sudo service clickhouse-server stop +sudo cp ClickHouse/build/dbms/programs/clickhouse /usr/bin/ +sudo service clickhouse-server start +``` + +Обратите внимание, что `clickhouse-client`, `clickhouse-server` и другие, являеются симлинками на общий бинарник `clickhouse`. + +Также вы можете запустить собранный вами ClickHouse с конфигурационным файлом системного ClickHouse: +``` +sudo service clickhouse-server stop +sudo -u clickhouse ClickHouse/build/dbms/programs/clickhouse server --config-file /etc/clickhouse-server/config.xml +``` + + +# Среда разработки + +Если вы не знаете, какую среду разработки использовать, то рекомендуется использовать CLion. CLion является платным ПО, но его можно использовать бесплатно в течение пробного периода. Также он бесплатен для учащихся. CLion можно использовать как под Linux, так и под Mac OS X. + +Также в качестве среды разработки, вы можете использовать KDevelop или QTCreator. KDevelop - очень удобная, но нестабильная среда разработки. Если KDevelop вылетает через небольшое время после открытия проекта, вам следует нажать на кнопку "Stop All" как только он открыл список файлов проекта. После этого, KDevelop можно будет использовать. + +В качестве простых редакторов кода можно использовать Sublime Text или Visual Studio Code или Kate (все варианты доступны под Linux). + +На всякий случай заметим, что CLion самостоятельно создаёт свою build директорию, самостоятельно выбирает тип сборки debug по-умолчанию, для конфигурации использует встроенную в CLion версию CMake вместо установленного вами, а для запуска задач использует make вместо ninja. Это нормально, просто имейте это ввиду, чтобы не возникало путаницы. + + +# Написание кода + +Описание архитектуры ClickHouse: https://clickhouse.yandex/docs/ru/development/architecture/ + +Стиль кода: https://clickhouse.yandex/docs/ru/development/style/ + +Разработка тестов: https://clickhouse.yandex/docs/ru/development/tests/ + +Список задач: https://github.com/ClickHouse/ClickHouse/blob/master/dbms/tests/instructions/easy_tasks_sorted_ru.md + + +# Тестовые данные + +Разработка ClickHouse часто требует загрузки реалистичных наборов данных. Особенно это важно для тестирования производительности. Специально для вас мы подготовили набор данных, представляющий собой анонимизированные данные Яндекс.Метрики. Загрузка этих данных потребует ещё 3 GB места на диске. Для выполнения большинства задач разработки, загружать эти данные не обязательно. + +``` +sudo apt install wget xz-utils + +wget https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_v1.tsv.xz +wget https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz + +xz -v -d hits_v1.tsv.xz +xz -v -d visits_v1.tsv.xz + +clickhouse-client + +CREATE TABLE test.hits ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime); + +CREATE TABLE test.visits ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), `Goals.ID` Array(UInt32), `Goals.Serial` Array(UInt32), `Goals.EventTime` Array(DateTime), `Goals.Price` Array(Int64), `Goals.OrderID` Array(String), `Goals.CurrencyID` Array(UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, `TraficSource.ID` Array(Int8), `TraficSource.SearchEngineID` Array(UInt16), `TraficSource.AdvEngineID` Array(UInt8), `TraficSource.PlaceID` Array(UInt16), `TraficSource.SocialSourceNetworkID` Array(UInt8), `TraficSource.Domain` Array(String), `TraficSource.SearchPhrase` Array(String), `TraficSource.SocialSourcePage` Array(String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `Market.Type` Array(UInt8), `Market.GoalID` Array(UInt32), `Market.OrderID` Array(String), `Market.OrderPrice` Array(Int64), `Market.PP` Array(UInt32), `Market.DirectPlaceID` Array(UInt32), `Market.DirectOrderID` Array(UInt32), `Market.DirectBannerID` Array(UInt32), `Market.GoodID` Array(String), `Market.GoodName` Array(String), `Market.GoodQuantity` Array(Int32), `Market.GoodPrice` Array(Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID); + +clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.hits FORMAT TSV" < hits_v1.tsv +clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.visits FORMAT TSV" < visits_v1.tsv +``` + + +# Создание pull request + +Откройте свой форк репозитория в интерфейсе GitHub. Если вы вели разработку в бранче, выберите этот бранч. На странице будет доступна кнопка "Pull request". По сути, это означает "создать заявку на принятие моих изменений в основной репозиторий". + +Pull request можно создать, даже если работа над задачей ещё не завершена. В этом случае, добавьте в его название слово "WIP" (work in progress). Название можно будет изменить позже. Это полезно для совместного просмотра и обсуждения изменений, а также для запуска всех имеющихся тестов. Введите краткое описание изменений - впоследствии, оно будет использовано для релизных changelog. + +Тесты будут запущены, как только сотрудники Яндекса поставят для pull request тег "Can be tested". Результаты первых проверок (стиль кода) появятся уже через несколько минут. Результаты сборки появятся примерно через пол часа. Результаты основного набора тестов будут доступны в пределах часа. + +Система подготовит сборки ClickHouse специально для вашего pull request. Для их получения, нажмите на ссылку "Details" у проверки "Clickhouse build check". Там вы сможете найти прямые ссылки на собранные .deb пакеты ClickHouse, которые, при желании, вы даже сможете установить на свои продакшен серверы (если не страшно). + +Вероятнее всего, часть сборок не будет успешной с первого раза. Ведь мы проверяем сборку кода и gcc и clang, а при сборке с помощью clang включаются почти все существующие в природе warnings (всегда с флагом `-Werror`). На той же странице, вы сможете найти логи сборки - вам не обязательно самому собирать ClickHouse всеми возможными способами. From 78f13839acd6f7f6d2560bc5793b1e0a4b17a935 Mon Sep 17 00:00:00 2001 From: akonyaev Date: Thu, 21 Nov 2019 18:46:25 +0300 Subject: [PATCH 07/42] Add exception for illegal types for conversion functions with -OrZero or -OrNull postfixes. Add docs for conversion functions with -OrZero and -OrNull postfixes. --- dbms/src/Functions/FunctionsConversion.h | 14 ++++++++-- .../functions/type_conversion_functions.md | 28 +++++++++++++++++++ .../functions/type_conversion_functions.md | 28 +++++++++++++++++++ 3 files changed, 68 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 0d058807a8b..8cca2768ff2 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -971,8 +971,18 @@ public: ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (!isStringOrFixedString(arguments[0].type)) - throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + { + if (this->getName().find("OrZero") != std::string::npos || + this->getName().find("OrNull") != std::string::npos) + throw Exception( + "Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() + + ". Conversion functions with postfix 'OrZero' or 'OrNull' should take String argument", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + else + throw Exception( + "Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } if (arguments.size() == 2) { diff --git a/docs/en/query_language/functions/type_conversion_functions.md b/docs/en/query_language/functions/type_conversion_functions.md index 7cca9e3fa1f..900bc8e0629 100644 --- a/docs/en/query_language/functions/type_conversion_functions.md +++ b/docs/en/query_language/functions/type_conversion_functions.md @@ -40,8 +40,36 @@ SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8) ## toInt(8|16|32|64)OrZero +It takes an argument of type String and tries to parse it into Int (8 | 16 | 32 | 64). If failed, returns 0. + +**Example** + +```sql +select toInt64OrZero('123123'), toInt8OrZero('123qwe123') +``` +```text +┌─toInt64OrZero('123123')─┬─toInt8OrZero('123qwe123')─┐ +│ 123123 │ 0 │ +└─────────────────────────┴───────────────────────────┘ +``` + + ## toInt(8|16|32|64)OrNull +It takes an argument of type String and tries to parse it into Int (8 | 16 | 32 | 64). If failed, returns NULL. + +**Example** + +```sql +select toInt64OrNull('123123'), toInt8OrNull('123qwe123') +``` +```text +┌─toInt64OrNull('123123')─┬─toInt8OrNull('123qwe123')─┐ +│ 123123 │ ᴺᵁᴸᴸ │ +└─────────────────────────┴───────────────────────────┘ +``` + + ## toUInt(8|16|32|64) Converts an input value to the [UInt](../../data_types/int_uint.md) data type. This function family includes: diff --git a/docs/ru/query_language/functions/type_conversion_functions.md b/docs/ru/query_language/functions/type_conversion_functions.md index af02eeae835..a94d96e7022 100644 --- a/docs/ru/query_language/functions/type_conversion_functions.md +++ b/docs/ru/query_language/functions/type_conversion_functions.md @@ -40,8 +40,36 @@ SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8) ## toInt(8|16|32|64)OrZero +Принимает аргумент типа String и пытается его распарсить в Int(8|16|32|64). Если не удалось - возвращает 0. + +**Пример** + +```sql +select toInt64OrZero('123123'), toInt8OrZero('123qwe123') +``` +```text +┌─toInt64OrZero('123123')─┬─toInt8OrZero('123qwe123')─┐ +│ 123123 │ 0 │ +└─────────────────────────┴───────────────────────────┘ +``` + + ## toInt(8|16|32|64)OrNull +Принимает аргумент типа String и пытается его распарсить в Int(8|16|32|64). Если не удалось - возвращает NULL. + +**Пример** + +```sql +select toInt64OrNull('123123'), toInt8OrNull('123qwe123') +``` +```text +┌─toInt64OrNull('123123')─┬─toInt8OrNull('123qwe123')─┐ +│ 123123 │ ᴺᵁᴸᴸ │ +└─────────────────────────┴───────────────────────────┘ +``` + + ## toUInt(8|16|32|64) Преобраует входное значение к типу [UInt](../../data_types/int_uint.md). Семейство функций включает: From 6834dd5fd0ed80db27c68f2d7a2a009ee95b3bfd Mon Sep 17 00:00:00 2001 From: akonyaev Date: Thu, 21 Nov 2019 18:53:14 +0300 Subject: [PATCH 08/42] Add exception for illegal types for conversion functions with -OrZero or -OrNull postfixes. Add docs for conversion functions with -OrZero and -OrNull postfixes. --- dbms/src/Functions/FunctionsConversion.h | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 8cca2768ff2..09a23f83414 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -974,13 +974,11 @@ public: { if (this->getName().find("OrZero") != std::string::npos || this->getName().find("OrNull") != std::string::npos) - throw Exception( - "Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() + + throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() + ". Conversion functions with postfix 'OrZero' or 'OrNull' should take String argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); else - throw Exception( - "Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName(), + throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } From 608aa276ab3737da9f3d9aca9f2da79bd6a763f7 Mon Sep 17 00:00:00 2001 From: Ding Xiang Fei Date: Mon, 11 Nov 2019 14:53:21 +0800 Subject: [PATCH 09/42] add safeguard against contention to check target partition row counts --- dbms/programs/copier/ClusterCopier.cpp | 428 +++++++++++++++++++------ 1 file changed, 322 insertions(+), 106 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 5fc1d76b542..95a2b2fcc69 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1,6 +1,7 @@ #include "ClusterCopier.h" #include +#include #include #include #include @@ -178,7 +179,9 @@ struct ShardPartition ShardPartition(TaskShard & parent, const String & name_quoted_) : task_shard(parent), name(name_quoted_) {} String getPartitionPath() const; + String getPartitionCleanStartPath() const; String getCommonPartitionIsDirtyPath() const; + String getCommonPartitionIsCleanedPath() const; String getPartitionActiveWorkersPath() const; String getActiveWorkerPath() const; String getPartitionShardsPath() const; @@ -259,6 +262,8 @@ struct TaskTable String getPartitionPath(const String & partition_name) const; String getPartitionIsDirtyPath(const String & partition_name) const; + String getPartitionIsCleanedPath(const String & partition_name) const; + String getPartitionTaskStatusPath(const String & partition_name) const; String name_in_config; @@ -369,23 +374,6 @@ struct MultiTransactionInfo Coordination::Responses responses; }; - -/// Atomically checks that is_dirty node is not exists, and made the remaining op -/// Returns relative number of failed operation in the second field (the passed op has 0 index) -static MultiTransactionInfo checkNoNodeAndCommit( - const zkutil::ZooKeeperPtr & zookeeper, - const String & checking_node_path, - Coordination::RequestPtr && op) -{ - MultiTransactionInfo info; - info.requests.emplace_back(zkutil::makeCreateRequest(checking_node_path, "", zkutil::CreateMode::Persistent)); - info.requests.emplace_back(zkutil::makeRemoveRequest(checking_node_path, -1)); - info.requests.emplace_back(std::move(op)); - info.code = zookeeper->tryMulti(info.requests, info.responses); - return info; -} - - // Creates AST representing 'ENGINE = Distributed(cluster, db, table, [sharding_key]) std::shared_ptr createASTStorageDistributed( const String & cluster_name, const String & database, const String & table, const ASTPtr & sharding_key_ast = nullptr) @@ -431,6 +419,11 @@ String TaskTable::getPartitionPath(const String & partition_name) const + "/" + escapeForFileName(partition_name); // 201701 } +String ShardPartition::getPartitionCleanStartPath() const +{ + return getPartitionPath() + "/clean_start"; +} + String ShardPartition::getPartitionPath() const { return task_shard.task_table.getPartitionPath(name); @@ -438,8 +431,9 @@ String ShardPartition::getPartitionPath() const String ShardPartition::getShardStatusPath() const { - // /root/table_test.hits/201701/1 - return getPartitionPath() + "/shards/" + toString(task_shard.numberInCluster()); + // schema: //tables///shards/ + // e.g. /root/table_test.hits/201701/shards/1 + return getPartitionShardsPath() + "/" + toString(task_shard.numberInCluster()); } String ShardPartition::getPartitionShardsPath() const @@ -462,11 +456,25 @@ String ShardPartition::getCommonPartitionIsDirtyPath() const return getPartitionPath() + "/is_dirty"; } +String ShardPartition::getCommonPartitionIsCleanedPath() const { + return getCommonPartitionIsDirtyPath() + "/cleaned"; +} + String TaskTable::getPartitionIsDirtyPath(const String & partition_name) const { return getPartitionPath(partition_name) + "/is_dirty"; } +String TaskTable::getPartitionIsCleanedPath(const String & partition_name) const +{ + return getPartitionIsDirtyPath(partition_name) + "/cleaned"; +} + +String TaskTable::getPartitionTaskStatusPath(const String & partition_name) const +{ + return getPartitionPath(partition_name) + "/shards"; +} + String DB::TaskShard::getDescription() const { std::stringstream ss; @@ -1129,9 +1137,9 @@ protected: } /** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock. - * State of some task could be changed during the processing. - * We have to ensure that all shards have the finished state and there are no dirty flag. - * Moreover, we have to check status twice and check zxid, because state could be changed during the checking. + * State of some task could change during the processing. + * We have to ensure that all shards have the finished state and there is no dirty flag. + * Moreover, we have to check status twice and check zxid, because state can change during the checking. */ bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) { @@ -1170,10 +1178,22 @@ protected: } // Check that partition is not dirty - if (zookeeper->exists(task_table.getPartitionIsDirtyPath(partition_name))) { - LOG_INFO(log, "Partition " << partition_name << " become dirty"); - return false; + CleanStateClock clean_state_clock ( + zookeeper, + task_table.getPartitionIsDirtyPath(partition_name), + task_table.getPartitionIsCleanedPath(partition_name) + ); + Coordination::Stat stat; + LogicalClock task_start_clock; + if (zookeeper->exists(task_table.getPartitionTaskStatusPath(partition_name), &stat)) + task_start_clock = LogicalClock(stat.mzxid); + zookeeper->get(task_table.getPartitionTaskStatusPath(partition_name), &stat); + if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) + { + LOG_INFO(log, "Partition " << partition_name << " become dirty"); + return false; + } } get_futures.clear(); @@ -1260,17 +1280,122 @@ protected: return res; } - bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper) + class LogicalClock { + public: + std::optional zxid; + + LogicalClock() = default; + + LogicalClock(UInt64 zxid) + : zxid(zxid) + {} + + bool hasHappened() const { + return bool(zxid); + } + + // happens-before relation with a reasonable time bound + bool happensBefore(const struct LogicalClock & other) const { + const UInt64 HALF = 1ull << 63; + return + !zxid || + (other.zxid && *zxid <= *other.zxid && *other.zxid - *zxid < HALF) || + (other.zxid && *zxid >= *other.zxid && *zxid - *other.zxid > HALF); + } + + bool operator<=(const struct LogicalClock & other) const { + return happensBefore(other); + } + + // strict equality check + bool operator==(const struct LogicalClock & other) const { + return zxid == other.zxid; + } + }; + + class CleanStateClock { + public: + LogicalClock discovery_zxid; + std::optional discovery_version; + + LogicalClock clean_state_zxid; + std::optional clean_state_version; + + std::shared_ptr stale; + + bool is_clean() const { + return + !is_stale() + && ( + !discovery_zxid.hasHappened() + || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); + } + + bool is_stale() const { + return stale->load(); + } + + CleanStateClock( + const zkutil::ZooKeeperPtr & zookeeper, + const String & discovery_path, + const String & clean_state_path) + : stale(std::make_shared(false)) + { + Coordination::Stat stat; + String _some_data; + auto watch_callback = + [stale = stale] (const Coordination::WatchResponse & rsp) { + auto logger = &Poco::Logger::get("ClusterCopier"); + if (rsp.error == Coordination::ZOK) + { + switch (rsp.type) + { + case Coordination::CREATED: + LOG_DEBUG(logger, "CleanStateClock change: CREATED, at " << rsp.path); + stale->store(true); + break; + case Coordination::CHANGED: + LOG_DEBUG(logger, "CleanStateClock change: CHANGED, at" << rsp.path); + stale->store(true); + } + } + }; + if (zookeeper->tryGetWatch(discovery_path, _some_data, &stat, watch_callback)) { + discovery_zxid = LogicalClock(stat.mzxid); + discovery_version = stat.version; + } + if (zookeeper->tryGetWatch(clean_state_path, _some_data, &stat, watch_callback)) { + clean_state_zxid = LogicalClock(stat.mzxid); + clean_state_version = stat.version; + } + } + + bool operator==(const struct CleanStateClock & other) const { + return !is_stale() + && !other.is_stale() + && discovery_zxid == other.discovery_zxid + && discovery_version == other.discovery_version + && clean_state_zxid == other.clean_state_zxid + && clean_state_version == other.clean_state_version; + } + + bool operator!=(const struct CleanStateClock & other) const { + return !(*this == other); + } + }; + + bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock) { if (is_safe_mode) throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED); TaskTable & task_table = task_partition.task_shard.task_table; - String current_shards_path = task_partition.getPartitionShardsPath(); - String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); - String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - String dirt_cleaner_path = is_dirty_flag_path + "/cleaner"; + const String current_shards_path = task_partition.getPartitionShardsPath(); + const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); + const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); + const String dirt_cleaner_path = is_dirty_flag_path + "/cleaner"; + const String is_dirt_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); zkutil::EphemeralNodeHolder::Ptr cleaner_holder; try @@ -1294,44 +1419,92 @@ protected: { if (stat.numChildren != 0) { - LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers, sleep"); + LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers while trying to drop it. Going to sleep."); std::this_thread::sleep_for(default_sleep_time); return false; } + else + { + zookeeper->remove(current_partition_active_workers_dir); + } } - /// Remove all status nodes - zookeeper->tryRemoveRecursive(current_shards_path); - - String query = "ALTER TABLE " + getQuotedTable(task_table.table_push); - query += " DROP PARTITION " + task_partition.name + ""; - - /// TODO: use this statement after servers will be updated up to 1.1.54310 - // query += " DROP PARTITION ID '" + task_partition.name + "'"; - - ClusterPtr & cluster_push = task_table.cluster_push; - Settings settings_push = task_cluster->settings_push; - - /// It is important, DROP PARTITION must be done synchronously - settings_push.replication_alter_partitions_sync = 2; - - LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); - /// Limit number of max executing replicas to 1 - UInt64 num_shards = executeQueryOnCluster(cluster_push, query, nullptr, &settings_push, PoolMode::GET_ONE, 1); - - if (num_shards < cluster_push->getShardCount()) { - LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); - return false; + zkutil::EphemeralNodeHolder::Ptr active_workers_lock; + try + { + active_workers_lock = zkutil::EphemeralNodeHolder::create(current_partition_active_workers_dir, *zookeeper, host_id); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::ZNODEEXISTS) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " is being filled now by somebody, sleep"); + return false; + } + + throw; + } + + // Lock the dirty flag + zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value()); + zookeeper->tryRemove(task_partition.getPartitionCleanStartPath()); + CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirt_cleaned_path); + + /// Remove all status nodes + { + Strings children; + if (zookeeper->tryGetChildren(current_shards_path, children) == Coordination::ZOK) + for (const auto & child : children) + { + zookeeper->removeRecursive(current_shards_path + "/" + child); + } + } + + String query = "ALTER TABLE " + getQuotedTable(task_table.table_push); + query += " DROP PARTITION " + task_partition.name + ""; + + /// TODO: use this statement after servers will be updated up to 1.1.54310 + // query += " DROP PARTITION ID '" + task_partition.name + "'"; + + ClusterPtr & cluster_push = task_table.cluster_push; + Settings settings_push = task_cluster->settings_push; + + /// It is important, DROP PARTITION must be done synchronously + settings_push.replication_alter_partitions_sync = 2; + + LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); + /// Limit number of max executing replicas to 1 + UInt64 num_shards = executeQueryOnCluster(cluster_push, query, nullptr, &settings_push, PoolMode::GET_ONE, 1); + + if (num_shards < cluster_push->getShardCount()) + { + LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); + return false; + } + + /// Update the locking node + if (!my_clock.is_stale()) + { + zookeeper->set(is_dirty_flag_path, host_id, my_clock.discovery_version.value()); + if (my_clock.clean_state_version) + zookeeper->set(is_dirt_cleaned_path, host_id, my_clock.clean_state_version.value()); + else + zookeeper->create(is_dirt_cleaned_path, host_id, zkutil::CreateMode::Persistent); + } + else + { + LOG_DEBUG(log, "Clean state is altered when dropping the partition, cowardly bailing"); + /// clean state is stale + return false; + } + + LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name); + if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS) + zookeeper->set(current_shards_path, host_id); } - /// Remove the locking node - Coordination::Requests requests; - requests.emplace_back(zkutil::makeRemoveRequest(dirt_cleaner_path, -1)); - requests.emplace_back(zkutil::makeRemoveRequest(is_dirty_flag_path, -1)); - zookeeper->multi(requests); - - LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name); + LOG_INFO(log, "Partition " << task_partition.name << " is safe for work now."); return true; } @@ -1362,6 +1535,7 @@ protected: /// Process each source shard having current partition and copy current partition /// NOTE: shards are sorted by "distance" to current host + bool has_shard_to_process = false; for (const TaskShardPtr & shard : task_table.all_shards) { /// Does shard have a node with current partition? @@ -1405,6 +1579,7 @@ protected: bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote; PartitionTaskStatus task_status = PartitionTaskStatus::Error; bool was_error = false; + has_shard_to_process = true; for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) { task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task); @@ -1432,11 +1607,13 @@ protected: cluster_partition.elapsed_time_seconds += watch.elapsedSeconds(); /// Check that whole cluster partition is done - /// Firstly check number failed partition tasks, than look into ZooKeeper and ensure that each partition is done + /// Firstly check the number of failed partition tasks, then look into ZooKeeper and ensure that each partition is done bool partition_is_done = num_failed_shards == 0; try { - partition_is_done = partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards); + partition_is_done = + !has_shard_to_process + || (partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards)); } catch (...) { @@ -1526,20 +1703,35 @@ protected: TaskTable & task_table = task_shard.task_table; ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); + /// We need to update table definitions for each partition, it could be changed after ALTER + createShardInternalTables(timeouts, task_shard); + auto zookeeper = context.getZooKeeper(); - String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - String current_task_is_active_path = task_partition.getActiveWorkerPath(); - String current_task_status_path = task_partition.getShardStatusPath(); + const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); + const String is_dirt_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); + const String current_task_is_active_path = task_partition.getActiveWorkerPath(); + const String current_task_status_path = task_partition.getShardStatusPath(); /// Auxiliary functions: /// Creates is_dirty node to initialize DROP PARTITION - auto create_is_dirty_node = [&] () + auto create_is_dirty_node = [&, this] (const CleanStateClock & clock) { - auto code = zookeeper->tryCreate(is_dirty_flag_path, current_task_status_path, zkutil::CreateMode::Persistent); - if (code && code != Coordination::ZNODEEXISTS) - throw Coordination::Exception(code, is_dirty_flag_path); + if (clock.is_stale()) + LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing"); + else if (!clock.is_clean()) + LOG_DEBUG(log, "Thank you, Captain Obvious"); + else if (clock.discovery_version) + { + LOG_DEBUG(log, "Updating clean state clock"); + zookeeper->set(is_dirty_flag_path, host_id, clock.discovery_version.value()); + } + else + { + LOG_DEBUG(log, "Creating clean state clock"); + zookeeper->create(is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent); + } }; /// Returns SELECT query filtering current partition and applying user filter @@ -1563,14 +1755,29 @@ protected: LOG_DEBUG(log, "Processing " << current_task_status_path); + CleanStateClock clean_state_clock (zookeeper, is_dirty_flag_path, is_dirt_cleaned_path); + + LogicalClock task_start_clock; + { + Coordination::Stat stat; + if (zookeeper->exists(task_partition.getPartitionShardsPath(), &stat)) + task_start_clock = LogicalClock(stat.mzxid); + } + /// Do not start if partition is dirty, try to clean it - if (zookeeper->exists(is_dirty_flag_path)) + if (clean_state_clock.is_clean() + && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " appears to be clean"); + zookeeper->createAncestors(current_task_status_path); + } + else { LOG_DEBUG(log, "Partition " << task_partition.name << " is dirty, try to drop it"); try { - tryDropPartition(task_partition, zookeeper); + tryDropPartition(task_partition, zookeeper, clean_state_clock); } catch (...) { @@ -1598,7 +1805,8 @@ protected: throw; } - /// Exit if task has been already processed, create blocking node if it is abandoned + /// Exit if task has been already processed; + /// create blocking node to signal cleaning up if it is abandoned { String status_data; if (zookeeper->tryGet(current_task_status_path, status_data)) @@ -1611,21 +1819,21 @@ protected: } // Task is abandoned, initialize DROP PARTITION - LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << status.owner); + LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << status.owner << ". Partition will be dropped and refilled."); - create_is_dirty_node(); + create_is_dirty_node(clean_state_clock); return PartitionTaskStatus::Error; } } - zookeeper->createAncestors(current_task_status_path); - - /// We need to update table definitions for each partition, it could be changed after ALTER - createShardInternalTables(timeouts, task_shard); - /// Check that destination partition is empty if we are first worker /// NOTE: this check is incorrect if pull and push tables have different partition key! + String clean_start_status; + if (!zookeeper->tryGet(task_partition.getPartitionCleanStartPath(), clean_start_status) || clean_start_status != "ok") { + zookeeper->createIfNotExists(task_partition.getPartitionCleanStartPath(), ""); + auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", *zookeeper, host_id); + // Maybe we are the first worker ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()"); UInt64 count; { @@ -1643,36 +1851,38 @@ protected: Coordination::Stat stat_shards; zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards); + /// NOTE: partition is still fresh if dirt discovery happens before cleaning if (stat_shards.numChildren == 0) { - LOG_WARNING(log, "There are no any workers for partition " << task_partition.name + LOG_WARNING(log, "There are no workers for partition " << task_partition.name << ", but destination table contains " << count << " rows" << ". Partition will be dropped and refilled."); - create_is_dirty_node(); + create_is_dirty_node(clean_state_clock); return PartitionTaskStatus::Error; } } + zookeeper->set(task_partition.getPartitionCleanStartPath(), "ok"); } + /// At this point, we need to sync that the destination table is clean + /// before any actual work /// Try start processing, create node about it { String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); - auto op_create = zkutil::makeCreateRequest(current_task_status_path, start_state, zkutil::CreateMode::Persistent); - MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_create)); - - if (info.code) + CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirt_cleaned_path); + if (clean_state_clock != new_clean_state_clock) { - zkutil::KeeperMultiException exception(info.code, info.requests, info.responses); - - if (exception.getPathForFirstFailedOp() == is_dirty_flag_path) - { - LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled"); - return PartitionTaskStatus::Error; - } - - throw exception; + LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); + return PartitionTaskStatus::Error; } + else if (!new_clean_state_clock.is_clean()) + { + LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled"); + create_is_dirty_node(new_clean_state_clock); + return PartitionTaskStatus::Error; + } + zookeeper->create(current_task_status_path, start_state, zkutil::CreateMode::Persistent); } /// Try create table (if not exists) on each shard @@ -1733,12 +1943,13 @@ protected: output = io_insert.out; } + /// Fail-fast optimization to abort copying when the current clean state expires std::future future_is_dirty_checker; Stopwatch watch(CLOCK_MONOTONIC_COARSE); constexpr UInt64 check_period_milliseconds = 500; - /// Will asynchronously check that ZooKeeper connection and is_dirty flag appearing while copy data + /// Will asynchronously check that ZooKeeper connection and is_dirty flag appearing while copying data auto cancel_check = [&] () { if (zookeeper->expired()) @@ -1754,7 +1965,13 @@ protected: Coordination::ExistsResponse status = future_is_dirty_checker.get(); if (status.error != Coordination::ZNONODE) + { + LogicalClock dirt_discovery_epoch (status.stat.mzxid); + if (dirt_discovery_epoch == clean_state_clock.discovery_zxid) { + return false; + } throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED); + } } return false; @@ -1789,20 +2006,19 @@ protected: /// Finalize the processing, change state of current partition task (and also check is_dirty flag) { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); - auto op_set = zkutil::makeSetRequest(current_task_status_path, state_finished, 0); - MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_set)); - - if (info.code) + CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirt_cleaned_path); + if (clean_state_clock != new_clean_state_clock) { - zkutil::KeeperMultiException exception(info.code, info.requests, info.responses); - - if (exception.getPathForFirstFailedOp() == is_dirty_flag_path) - LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled"); - else - LOG_INFO(log, "Someone made the node abandoned. Will refill partition. " << zkutil::ZooKeeper::error2string(info.code)); - + LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); return PartitionTaskStatus::Error; } + else if (!new_clean_state_clock.is_clean()) + { + LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled"); + create_is_dirty_node(new_clean_state_clock); + return PartitionTaskStatus::Error; + } + zookeeper->set(current_task_status_path, state_finished, 0); } LOG_INFO(log, "Partition " << task_partition.name << " copied"); From 8330a095757853f92732b292828c87f8081da279 Mon Sep 17 00:00:00 2001 From: Ding Xiang Fei Date: Tue, 19 Nov 2019 10:24:41 +0800 Subject: [PATCH 10/42] apply style guide and fix CI warnings --- dbms/programs/copier/ClusterCopier.cpp | 49 ++++++++++++++++---------- 1 file changed, 31 insertions(+), 18 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 95a2b2fcc69..066176354b7 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -456,7 +456,8 @@ String ShardPartition::getCommonPartitionIsDirtyPath() const return getPartitionPath() + "/is_dirty"; } -String ShardPartition::getCommonPartitionIsCleanedPath() const { +String ShardPartition::getCommonPartitionIsCleanedPath() const +{ return getCommonPartitionIsDirtyPath() + "/cleaned"; } @@ -1280,22 +1281,25 @@ protected: return res; } - class LogicalClock { + class LogicalClock + { public: std::optional zxid; LogicalClock() = default; - LogicalClock(UInt64 zxid) - : zxid(zxid) + LogicalClock(UInt64 _zxid) + : zxid(_zxid) {} - bool hasHappened() const { + bool hasHappened() const + { return bool(zxid); } // happens-before relation with a reasonable time bound - bool happensBefore(const struct LogicalClock & other) const { + bool happensBefore(const LogicalClock & other) const + { const UInt64 HALF = 1ull << 63; return !zxid || @@ -1303,17 +1307,20 @@ protected: (other.zxid && *zxid >= *other.zxid && *zxid - *other.zxid > HALF); } - bool operator<=(const struct LogicalClock & other) const { + bool operator<=(const LogicalClock & other) const + { return happensBefore(other); } // strict equality check - bool operator==(const struct LogicalClock & other) const { + bool operator==(const LogicalClock & other) const + { return zxid == other.zxid; } }; - class CleanStateClock { + class CleanStateClock + { public: LogicalClock discovery_zxid; std::optional discovery_version; @@ -1323,7 +1330,8 @@ protected: std::shared_ptr stale; - bool is_clean() const { + bool is_clean() const + { return !is_stale() && ( @@ -1331,7 +1339,8 @@ protected: || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); } - bool is_stale() const { + bool is_stale() const + { return stale->load(); } @@ -1344,7 +1353,8 @@ protected: Coordination::Stat stat; String _some_data; auto watch_callback = - [stale = stale] (const Coordination::WatchResponse & rsp) { + [stale = stale] (const Coordination::WatchResponse & rsp) + { auto logger = &Poco::Logger::get("ClusterCopier"); if (rsp.error == Coordination::ZOK) { @@ -1360,17 +1370,20 @@ protected: } } }; - if (zookeeper->tryGetWatch(discovery_path, _some_data, &stat, watch_callback)) { + if (zookeeper->tryGetWatch(discovery_path, _some_data, &stat, watch_callback)) + { discovery_zxid = LogicalClock(stat.mzxid); discovery_version = stat.version; } - if (zookeeper->tryGetWatch(clean_state_path, _some_data, &stat, watch_callback)) { + if (zookeeper->tryGetWatch(clean_state_path, _some_data, &stat, watch_callback)) + { clean_state_zxid = LogicalClock(stat.mzxid); clean_state_version = stat.version; } } - bool operator==(const struct CleanStateClock & other) const { + bool operator==(const CleanStateClock & other) const + { return !is_stale() && !other.is_stale() && discovery_zxid == other.discovery_zxid @@ -1379,7 +1392,8 @@ protected: && clean_state_version == other.clean_state_version; } - bool operator!=(const struct CleanStateClock & other) const { + bool operator!=(const CleanStateClock & other) const + { return !(*this == other); } }; @@ -1967,9 +1981,8 @@ protected: if (status.error != Coordination::ZNONODE) { LogicalClock dirt_discovery_epoch (status.stat.mzxid); - if (dirt_discovery_epoch == clean_state_clock.discovery_zxid) { + if (dirt_discovery_epoch == clean_state_clock.discovery_zxid) return false; - } throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED); } } From 1e2e7b41da25856ea0346244a5977a653151f599 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Fri, 22 Nov 2019 12:59:20 +0300 Subject: [PATCH 11/42] Updated ToC. Symlinks for fa/zh developer instructions --- docs/fa/development/developer_instruction.md | 1 + docs/toc_en.yml | 1 + docs/toc_fa.yml | 1 + docs/toc_ru.yml | 1 + docs/toc_zh.yml | 1 + docs/zh/development/developer_instruction.md | 1 + 6 files changed, 6 insertions(+) create mode 120000 docs/fa/development/developer_instruction.md create mode 120000 docs/zh/development/developer_instruction.md diff --git a/docs/fa/development/developer_instruction.md b/docs/fa/development/developer_instruction.md new file mode 120000 index 00000000000..bdfa9047aa2 --- /dev/null +++ b/docs/fa/development/developer_instruction.md @@ -0,0 +1 @@ +../../en/development/developer_instruction.md \ No newline at end of file diff --git a/docs/toc_en.yml b/docs/toc_en.yml index 356a256e2d0..3c4a5506a06 100644 --- a/docs/toc_en.yml +++ b/docs/toc_en.yml @@ -218,6 +218,7 @@ nav: - 'How to Build ClickHouse on Linux for Mac OS X': 'development/build_cross.md' - 'How to Write C++ code': 'development/style.md' - 'How to Run ClickHouse Tests': 'development/tests.md' + - 'The Beginner ClickHouse Developer Instruction': 'development/developer_instruction.md' - 'Third-Party Libraries Used': 'development/contrib.md' - 'What''s New': diff --git a/docs/toc_fa.yml b/docs/toc_fa.yml index 6457c2da42e..207034a8718 100644 --- a/docs/toc_fa.yml +++ b/docs/toc_fa.yml @@ -216,6 +216,7 @@ nav: - 'How to Build ClickHouse on Mac OS X': 'development/build_osx.md' - 'How to Write C++ code': 'development/style.md' - 'How to Run ClickHouse Tests': 'development/tests.md' + - 'The Beginner ClickHouse Developer Instruction': 'development/developer_instruction.md' - 'Third-Party Libraries Used': 'development/contrib.md' - 'What''s New': diff --git a/docs/toc_ru.yml b/docs/toc_ru.yml index fa2af4ce631..5c71894c8f6 100644 --- a/docs/toc_ru.yml +++ b/docs/toc_ru.yml @@ -217,6 +217,7 @@ nav: - 'Как собрать ClickHouse на Linux для Mac OS X': 'development/build_cross.md' - 'Как писать код на C++': 'development/style.md' - 'Как запустить тесты': 'development/tests.md' + - 'Инструкция для начинающего разработчика ClickHouse': 'development/developer_instruction.md' - 'Сторонние библиотеки': 'development/contrib.md' - 'Что нового': diff --git a/docs/toc_zh.yml b/docs/toc_zh.yml index 0dc022af1f9..e59f231654a 100644 --- a/docs/toc_zh.yml +++ b/docs/toc_zh.yml @@ -215,6 +215,7 @@ nav: - 'How to Build ClickHouse on Linux for Mac OS X': 'development/build_cross.md' - '如何编写C++代码': 'development/style.md' - '如何运行ClickHouse测试': 'development/tests.md' + - 'The Beginner ClickHouse Developer Instruction': 'development/developer_instruction.md' - '使用的第三方库': 'development/contrib.md' - '新功能特性': diff --git a/docs/zh/development/developer_instruction.md b/docs/zh/development/developer_instruction.md new file mode 120000 index 00000000000..bdfa9047aa2 --- /dev/null +++ b/docs/zh/development/developer_instruction.md @@ -0,0 +1 @@ +../../en/development/developer_instruction.md \ No newline at end of file From 6f80b0fed4d9d7d9cf0cc8b227d754322b9bdda2 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 22 Nov 2019 13:12:24 +0300 Subject: [PATCH 12/42] move specific functions to ReadHelpers --- dbms/src/IO/ReadHelpers.h | 32 +++++++++++++++ .../Impl/TabSeparatedRowInputFormat.cpp | 39 ------------------- 2 files changed, 32 insertions(+), 39 deletions(-) diff --git a/dbms/src/IO/ReadHelpers.h b/dbms/src/IO/ReadHelpers.h index e53346c606e..8bfd605cc83 100644 --- a/dbms/src/IO/ReadHelpers.h +++ b/dbms/src/IO/ReadHelpers.h @@ -924,4 +924,36 @@ if (method == DB::CompressionMethod::Gzip) return std::make_unique(args...); } +/// TODO (akuzm) - write comments for this and next function. +void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current) +{ + assert(current >= in.position()); + assert(current <= in.buffer().end()); + + const int old_bytes = memory.size(); + const int additional_bytes = current - in.position(); + const int new_bytes = old_bytes + additional_bytes; + /// There are no new bytes to add to memory. + /// No need to do extra stuff. + if (new_bytes == 0) + return; + memory.resize(new_bytes); + memcpy(memory.data() + old_bytes, in.position(), additional_bytes); + in.position() = current; +} + +bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current) +{ + assert(current <= in.buffer().end()); + + if (current < in.buffer().end()) + return true; + + saveUpToPosition(in, memory, current); + bool loaded_more = !in.eof(); + assert(in.position() == in.buffer().begin()); + current = in.position(); + return loaded_more; +} + } diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 3f8c64344f9..744b253fac5 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -384,39 +384,6 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) } } -void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current) -{ - assert(current >= in.position()); - assert(current <= in.buffer().end()); - - const int old_bytes = memory.size(); - const int additional_bytes = current - in.position(); - const int new_bytes = old_bytes + additional_bytes; - /// There are no new bytes to add to memory. - /// No need to do extra stuff. - if (new_bytes == 0) - return; - memory.resize(new_bytes); - memcpy(memory.data() + old_bytes, in.position(), additional_bytes); - in.position() = current; -} - -bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current) -{ - assert(current <= in.buffer().end()); - - if (current < in.buffer().end()) - { - return true; - } - - saveUpToPosition(in, memory, current); - bool loaded_more = !in.eof(); - assert(in.position() == in.buffer().begin()); - current = in.position(); - return loaded_more; -} - bool fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) { bool need_more_data = true; @@ -425,24 +392,18 @@ bool fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memo { pos = find_first_symbols<'\\', '\r', '\n'>(pos, in.buffer().end()); if (pos == in.buffer().end()) - { continue; - } if (*pos == '\\') { ++pos; if (loadAtPosition(in, memory, pos)) - { ++pos; - } } else if (*pos == '\n' || *pos == '\r') { if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size) - { need_more_data = false; - } ++pos; } From b29054270f9e9e0dc4b0f15ceb72985fe664de8b Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Fri, 22 Nov 2019 13:13:13 +0300 Subject: [PATCH 13/42] Slighly better wording --- docs/en/development/developer_instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/developer_instruction.md b/docs/en/development/developer_instruction.md index 463f186fb3b..0e4bfb62550 100644 --- a/docs/en/development/developer_instruction.md +++ b/docs/en/development/developer_instruction.md @@ -216,7 +216,7 @@ KDevelop and QTCreator are another great alternatives of an IDE for developing C As simple code editors you can use Sublime Text or Visual Studio Code, or Kate (all of which are available on Linux). -Just in case, it is worth mentioning that CLion creates by itself its own `build` path, it also selects by itself `debug` for build type, for configuration it uses a version of CMake that is defined in CLion and not the one installed by you, and finally CLion will use `make` to run build tasks instead of `ninja`. This is a normal behaviour, just keep that in mind to avoid confusion. +Just in case, it is worth mentioning that CLion creates `build` path on its own, it also on its own selects `debug` for build type, for configuration it uses a version of CMake that is defined in CLion and not the one installed by you, and finally CLion will use `make` to run build tasks instead of `ninja`. This is a normal behaviour, just keep that in mind to avoid confusion. # Writing Code From d495e282c70b9cc17646efacbf323f0edacbdb70 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 22 Nov 2019 13:38:08 +0300 Subject: [PATCH 14/42] returning back JSON and CSV --- dbms/src/Formats/FormatFactory.cpp | 4 ++ .../Formats/Impl/CSVRowInputFormat.cpp | 61 ++++++++++++++++++ .../Impl/JSONEachRowRowInputFormat.cpp | 64 +++++++++++++++++++ .../Impl/TabSeparatedRowInputFormat.cpp | 4 +- 4 files changed, 132 insertions(+), 1 deletion(-) diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index bd2544d7afd..57071b17c28 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -292,6 +292,8 @@ void registerOutputFormatProcessorTemplate(FormatFactory &factory); /// File Segmentation Engines for parallel reading void registerFileSegmentationEngineTabSeparated(FormatFactory & factory); +void registerFileSegmentationEngineCSV(FormatFactory & factory); +void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); /// Output only (presentational) formats. @@ -344,6 +346,8 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorTemplate(*this); registerFileSegmentationEngineTabSeparated(*this); + registerFileSegmentationEngineCSV(*this); + registerFileSegmentationEngineJSONEachRow(*this); registerOutputFormatNull(*this); diff --git a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index a13bb365192..9a7d13f3374 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -422,4 +422,65 @@ void registerInputFormatProcessorCSV(FormatFactory & factory) } } +bool fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) +{ + skipWhitespacesAndTabs(in); + + char * pos = in.position(); + bool quotes = false; + bool need_more_data = true; + + while (loadAtPosition(in, memory, pos) && need_more_data) + { + if (quotes) + { + pos = find_first_symbols<'"'>(pos, in.buffer().end()); + if (pos == in.buffer().end()) + continue; + if (*pos == '"') + { + ++pos; + if (loadAtPosition(in, memory, pos) && *pos == '"') + ++pos; + else + quotes = false; + } + } + else + { + pos = find_first_symbols<'"','\r', '\n'>(pos, in.buffer().end()); + if (pos == in.buffer().end()) + continue; + if (*pos == '"') + { + quotes = true; + ++pos; + } + else if (*pos == '\n') + { + if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size) + need_more_data = false; + ++pos; + if (loadAtPosition(in, memory, pos) && *pos == '\r') + ++pos; + } + else if (*pos == '\r') + { + if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size) + need_more_data = false; + ++pos; + if (loadAtPosition(in, memory, pos) && *pos == '\n') + ++pos; + } + } + } + loadAtPosition(in, memory, pos); + return true; +} + +void registerFileSegmentationEngineCSV(FormatFactory & factory) +{ + factory.registerFileSegmentationEngine("CSV", &fileSegmentationEngineCSVImpl); +} + } diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index a4a6911f8eb..ef5f3c0435c 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -270,4 +270,68 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory) }); } +bool fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) +{ + skipWhitespaceIfAny(in); + + char * pos = in.position(); + size_t balance = 0; + bool quotes = false; + + while (loadAtPosition(in, memory, pos) && (balance || memory.size() + static_cast(pos - in.position()) < min_chunk_size)) + { + if (quotes) + { + pos = find_first_symbols<'\\', '"'>(pos, in.buffer().end()); + if (pos == in.buffer().end()) + continue; + if (*pos == '\\') + { + ++pos; + if (loadAtPosition(in, memory, pos)) + ++pos; + } + else if (*pos == '"') + { + ++pos; + quotes = false; + } + } + else + { + pos = find_first_symbols<'{', '}', '\\', '"'>(pos, in.buffer().end()); + if (pos == in.buffer().end()) + continue; + if (*pos == '{') + { + ++balance; + ++pos; + } + else if (*pos == '}') + { + --balance; + ++pos; + } + else if (*pos == '\\') + { + ++pos; + if (loadAtPosition(in, memory, pos)) + ++pos; + } + else if (*pos == '"') + { + quotes = true; + ++pos; + } + } + } + loadAtPosition(in, memory, pos) + return true; +} + +void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory) +{ + factory.registerFileSegmentationEngine("JSONEachRow", &fileSegmentationEngineJSONEachRowImpl); +} + } diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 744b253fac5..edb5e26e026 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -388,9 +388,11 @@ bool fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memo { bool need_more_data = true; char * pos = in.position(); + while (loadAtPosition(in, memory, pos) && need_more_data) { pos = find_first_symbols<'\\', '\r', '\n'>(pos, in.buffer().end()); + if (pos == in.buffer().end()) continue; @@ -404,10 +406,10 @@ bool fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memo { if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size) need_more_data = false; - ++pos; } } + saveUpToPosition(in, memory, pos); return loadAtPosition(in, memory, pos); From 21cf4fd35e79b448caf4b2a4bb03736858b5cf38 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Fri, 22 Nov 2019 14:21:21 +0300 Subject: [PATCH 15/42] Updated ToC in Chinese --- docs/toc_zh.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/toc_zh.yml b/docs/toc_zh.yml index e59f231654a..c8be2ad7606 100644 --- a/docs/toc_zh.yml +++ b/docs/toc_zh.yml @@ -212,10 +212,10 @@ nav: - 'ClickHouse架构概述': 'development/architecture.md' - '如何在Linux中编译ClickHouse': 'development/build.md' - '如何在Mac OS X中编译ClickHouse': 'development/build_osx.md' - - 'How to Build ClickHouse on Linux for Mac OS X': 'development/build_cross.md' + - '如何在Linux中编译Mac OS X ClickHouse': 'development/build_cross.md' - '如何编写C++代码': 'development/style.md' - '如何运行ClickHouse测试': 'development/tests.md' - - 'The Beginner ClickHouse Developer Instruction': 'development/developer_instruction.md' + - '开发者指南': 'development/developer_instruction.md' - '使用的第三方库': 'development/contrib.md' - '新功能特性': From ffa3378fcda40115c90fe1c49ff93e57772ece83 Mon Sep 17 00:00:00 2001 From: elenaspb2019 Date: Mon, 18 Nov 2019 23:44:30 +0300 Subject: [PATCH 16/42] docs(bitTest, bitTestAny, bitTestAll):The new description of `bitTest`, `bitTestAny`, `bitTestAll` functions were added. --- .../query_language/functions/bit_functions.md | 177 ++++++++++++++++- .../query_language/functions/bit_functions.md | 179 ++++++++++++++++++ 2 files changed, 353 insertions(+), 3 deletions(-) diff --git a/docs/en/query_language/functions/bit_functions.md b/docs/en/query_language/functions/bit_functions.md index c08a80e2bbf..3c36a1b28bc 100644 --- a/docs/en/query_language/functions/bit_functions.md +++ b/docs/en/query_language/functions/bit_functions.md @@ -20,12 +20,183 @@ The result type is an integer with bits equal to the maximum bits of its argumen ## bitRotateRight(a, b) -## bitTest(a, b) +## bitTest {#bittest} -## bitTestAll(a, b) +Takes any integer and converts it into [binary form](https://en.wikipedia.org/wiki/Binary_number), returns the value of a bit at specified position. The countdown starts from 0 from the right to the left. -## bitTestAny(a, b) +**Syntax** +```sql +SELECT bitTest(number, index) +``` +**Parameters** + +- `number` – integer number. +- `index` – position of bit. + +**Returned values** + +Returns a value of bit at specified position. + +Type: `UInt8`. + +**Example** + +For example, the number 43 in base-2 (binary) numeral system is 101011. + +Query: + +```sql +SELECT bitTest(43, 1) +``` + +Result: + +```text +┌─bitTest(43, 1)─┐ +│ 1 │ +└────────────────┘ +``` + +Another example: + +Query: + +```sql +SELECT bitTest(43, 2) +``` + +Result: + +```text +┌─bitTest(43, 2)─┐ +│ 0 │ +└────────────────┘ +``` + +## bitTestAll {#bittestall} + +Returns result of [logical conjuction](https://en.wikipedia.org/wiki/Logical_conjunction) (AND operator) of all bits at given positions. The countdown starts from 0 from the right to the left. + +The conjuction for bitwise operations: + +0 AND 0 = 0 +0 AND 1 = 0 +1 AND 0 = 0 +1 AND 1 = 1 + +**Syntax** + +```sql +SELECT bitTestAll(number, index1, index2, index3, index4, ...) +``` + +**Parameters** + +- `number` – integer number. +- `index1`, `index2`, `index3`, `index4` – positions of bit. For example, for set of positions (`index1`, `index2`, `index3`, `index4`) is true if and only if all of its positions are true (`index1` ⋀ `index2`, ⋀ `index3` ⋀ `index4`). + +**Returned values** + +Returns result of logical conjuction. + +Type: `UInt8`. + +**Example** + +For example, the number 43 in base-2 (binary) numeral system is 101011. + +Query: + +```sql +SELECT bitTestAll(43, 0, 1, 3, 5) +``` + +Result: + +```text +┌─bitTestAll(43, 0, 1, 3, 5)─┐ +│ 1 │ +└────────────────────────────┘ +``` + +Another example: + +Query: + +```sql +SELECT bitTestAll(43, 0, 1, 3, 5, 2) +``` + +Result: + +```text +┌─bitTestAll(43, 0, 1, 3, 5, 2)─┐ +│ 0 │ +└───────────────────────────────┘ +``` + +## bitTestAny {#bittestany} + +Returns result of [logical disjunction](https://en.wikipedia.org/wiki/Logical_disjunction) (OR operator) of all bits at given positions. The countdown starts from 0 from the right to the left. + +The disjunction for bitwise operations: + +0 OR 0 = 0 +0 OR 1 = 1 +1 OR 0 = 1 +1 OR 1 = 1 + +**Syntax** + +```sql +SELECT bitTestAny(number, index1, index2, index3, index4, ...) +``` + +**Parameters** + +- `number` – integer number. +- `index1`, `index2`, `index3`, `index4` – positions of bit. + +**Returned values** + +Returns result of logical disjuction. + +Type: `UInt8`. + +**Example** + +For example, the number 43 in base-2 (binary) numeral system is 101011. + +Query: + +```sql +SELECT bitTestAny(43, 0, 2) +``` + +Result: + +```text +┌─bitTestAny(43, 0, 2)─┐ +│ 1 │ +└──────────────────────┘ +``` + +Another example: + +Query: + +```sql +SELECT bitTestAny(43, 4, 2) +``` + +Result: + +```text +┌─bitTestAny(43, 4, 2)─┐ +│ 0 │ +└──────────────────────┘ +``` [Original article](https://clickhouse.yandex/docs/en/query_language/functions/bit_functions/) diff --git a/docs/ru/query_language/functions/bit_functions.md b/docs/ru/query_language/functions/bit_functions.md index 7ba32ad6ba8..1921ad00324 100644 --- a/docs/ru/query_language/functions/bit_functions.md +++ b/docs/ru/query_language/functions/bit_functions.md @@ -16,4 +16,183 @@ ## bitShiftRight(a, b) +## bitTest {#bittest} + +Принимает любое целое число и конвертирует его в [двоичное число](https://en.wikipedia.org/wiki/Binary_number), возвращает значение бита в указанной позиции. Отсчет начинается с 0 справа налево. + +**Синтаксис** + +```sql +SELECT bitTest(number, index) +``` + +**Параметры** + +- `number` – целое число. +- `index` – position of bit. + +**Возвращаемое значение** + +Returns a value of bit at specified position. + +Тип: `UInt8`. + +**Пример** + +Например, число 43 в двоичной системе счисления равно: 101011. + +Запрос: + +```sql +SELECT bitTest(43, 1) +``` + +Ответ: + +```text +┌─bitTest(43, 1)─┐ +│ 1 │ +└────────────────┘ +``` + +Другой пример: + +Запрос: + +```sql +SELECT bitTest(43, 2) +``` + +Ответ: + +```text +┌─bitTest(43, 2)─┐ +│ 0 │ +└────────────────┘ +``` + +## bitTestAll {#bittestall} + +Возвращает результат [логической конъюнкции](https://en.wikipedia.org/wiki/Logical_conjunction) (оператор AND) всех битов в указанных позициях. Отсчет начинается с 0 справа налево. + +Бинарная конъюнкция: + +0 AND 0 = 0 +0 AND 1 = 0 +1 AND 0 = 0 +1 AND 1 = 1 + +**Синтаксис** + +```sql +SELECT bitTestAll(number, index1, index2, index3, index4, ...) +``` + +**Параметры** + +- `number` – целое число. +- `index1`, `index2`, `index3`, `index4` – позиция бита. Например, конъюнкция для набора позиций `index1`, `index2`, `index3`, `index4` является истинной, если все его позиции истинны `index1` ⋀ `index2` ⋀ `index3` ⋀ `index4`. + +**Возвращаемое значение** + +Возвращает результат логической конъюнкции. + +Тип: `UInt8`. + +**Пример** + +Например, число 43 в двоичной системе счисления равно: 101011. + +Запрос: + +```sql +SELECT bitTestAll(43, 0, 1, 3, 5) +``` + +Ответ: + +```text +┌─bitTestAll(43, 0, 1, 3, 5)─┐ +│ 1 │ +└────────────────────────────┘ +``` + +Другой пример: + +Запрос: + +```sql +SELECT bitTestAll(43, 0, 1, 3, 5, 2) +``` + +Ответ: + +```text +┌─bitTestAll(43, 0, 1, 3, 5, 2)─┐ +│ 0 │ +└───────────────────────────────┘ +``` + +## bitTestAny {#bittestany} + +Возвращает результат [логической дизъюнкции](https://en.wikipedia.org/wiki/Logical_disjunction) (оператор OR) всех битов в указанных позициях. Отсчет начинается с 0 справа налево. + +Бинарная дизъюнкция: + +0 OR 0 = 0 +0 OR 1 = 1 +1 OR 0 = 1 +1 OR 1 = 1 + +**Синтаксис** + +```sql +SELECT bitTestAny(number, index1, index2, index3, index4, ...) +``` + +**Параметры** + +- `number` – целое число. +- `index1`, `index2`, `index3`, `index4` – позиции бита. + +**Возвращаемое значение** + +Возвращает результат логической дизъюнкции. + +Тип: `UInt8`. + +**Пример** + +Например, число 43 в двоичной системе счисления равно: 101011. + +Запрос: + +```sql +SELECT bitTestAny(43, 0, 2) +``` + +Ответ: + +```text +┌─bitTestAny(43, 0, 2)─┐ +│ 1 │ +└──────────────────────┘ +``` + +Другой пример: + +Запрос: + +```sql +SELECT bitTestAny(43, 4, 2) +``` + +Ответ: + +```text +┌─bitTestAny(43, 4, 2)─┐ +│ 0 │ +└──────────────────────┘ +``` + [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/bit_functions/) From 05e05a709e44084d1c7f37864d2247e526e4158c Mon Sep 17 00:00:00 2001 From: Sergei Bocharov Date: Fri, 22 Nov 2019 14:53:56 +0300 Subject: [PATCH 17/42] Fixes --- docs/ru/query_language/functions/bit_functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/query_language/functions/bit_functions.md b/docs/ru/query_language/functions/bit_functions.md index 1921ad00324..e4af747fa47 100644 --- a/docs/ru/query_language/functions/bit_functions.md +++ b/docs/ru/query_language/functions/bit_functions.md @@ -33,7 +33,7 @@ SELECT bitTest(number, index) **Возвращаемое значение** -Returns a value of bit at specified position. +Возвращает значение бита в указанной позиции. Тип: `UInt8`. From 5df6878de652f9d745d91f6e21366a6ddebae3f0 Mon Sep 17 00:00:00 2001 From: andrei-karpliuk Date: Fri, 22 Nov 2019 16:02:33 +0300 Subject: [PATCH 18/42] fix incorrect Note formatting --- docs/en/query_language/system.md | 2 +- docs/ru/query_language/system.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/query_language/system.md b/docs/en/query_language/system.md index 74337052a82..229532d0f5e 100644 --- a/docs/en/query_language/system.md +++ b/docs/en/query_language/system.md @@ -95,7 +95,7 @@ Provides possibility to stop background merges for tables in the MergeTree famil ```sql SYSTEM STOP MERGES [[db.]merge_tree_family_table_name] ``` -!!! note "Note": +!!! note "Note" `DETACH / ATTACH` table will start background merges for the table even in case when merges have been stopped for all MergeTree tables before. diff --git a/docs/ru/query_language/system.md b/docs/ru/query_language/system.md index 7b94dd1eb69..3457a87e188 100644 --- a/docs/ru/query_language/system.md +++ b/docs/ru/query_language/system.md @@ -91,7 +91,7 @@ SYSTEM START DISTRIBUTED SENDS [db.] ```sql SYSTEM STOP MERGES [[db.]merge_tree_family_table_name] ``` -!!! note "Note": +!!! note "Note" `DETACH / ATTACH` таблицы восстанавливает фоновые мержи для этой таблицы (даже в случае отключения фоновых мержей для всех таблиц семейства MergeTree до `DETACH`). From 22a8e8efa7e5dcfc357a68e2fb1c337c6b47fcd6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 22 Nov 2019 16:53:26 +0300 Subject: [PATCH 19/42] better segmenating + move read functions to avoid linker errors --- dbms/src/IO/ReadHelpers.cpp | 32 +++++++++++++++++++ dbms/src/IO/ReadHelpers.h | 32 ++----------------- .../Formats/Impl/CSVRowInputFormat.cpp | 7 ++-- .../Impl/JSONEachRowRowInputFormat.cpp | 5 +-- 4 files changed, 40 insertions(+), 36 deletions(-) diff --git a/dbms/src/IO/ReadHelpers.cpp b/dbms/src/IO/ReadHelpers.cpp index 7c0c2301c28..860ce734c4c 100644 --- a/dbms/src/IO/ReadHelpers.cpp +++ b/dbms/src/IO/ReadHelpers.cpp @@ -1053,4 +1053,36 @@ void skipToUnescapedNextLineOrEOF(ReadBuffer & buf) } } +/// TODO (akuzm) - write comments for this and next function. +void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current) +{ + assert(current >= in.position()); + assert(current <= in.buffer().end()); + + const int old_bytes = memory.size(); + const int additional_bytes = current - in.position(); + const int new_bytes = old_bytes + additional_bytes; + /// There are no new bytes to add to memory. + /// No need to do extra stuff. + if (new_bytes == 0) + return; + memory.resize(new_bytes); + memcpy(memory.data() + old_bytes, in.position(), additional_bytes); + in.position() = current; +} + +bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current) +{ + assert(current <= in.buffer().end()); + + if (current < in.buffer().end()) + return true; + + saveUpToPosition(in, memory, current); + bool loaded_more = !in.eof(); + assert(in.position() == in.buffer().begin()); + current = in.position(); + return loaded_more; +} + } diff --git a/dbms/src/IO/ReadHelpers.h b/dbms/src/IO/ReadHelpers.h index 8bfd605cc83..4b162588d77 100644 --- a/dbms/src/IO/ReadHelpers.h +++ b/dbms/src/IO/ReadHelpers.h @@ -925,35 +925,7 @@ return std::make_unique(args...); } /// TODO (akuzm) - write comments for this and next function. -void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current) -{ - assert(current >= in.position()); - assert(current <= in.buffer().end()); - - const int old_bytes = memory.size(); - const int additional_bytes = current - in.position(); - const int new_bytes = old_bytes + additional_bytes; - /// There are no new bytes to add to memory. - /// No need to do extra stuff. - if (new_bytes == 0) - return; - memory.resize(new_bytes); - memcpy(memory.data() + old_bytes, in.position(), additional_bytes); - in.position() = current; -} - -bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current) -{ - assert(current <= in.buffer().end()); - - if (current < in.buffer().end()) - return true; - - saveUpToPosition(in, memory, current); - bool loaded_more = !in.eof(); - assert(in.position() == in.buffer().begin()); - current = in.position(); - return loaded_more; -} +void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current); +bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current); } diff --git a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 9a7d13f3374..502504b4ffd 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -424,8 +424,6 @@ void registerInputFormatProcessorCSV(FormatFactory & factory) bool fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) { - skipWhitespacesAndTabs(in); - char * pos = in.position(); bool quotes = false; bool need_more_data = true; @@ -474,8 +472,9 @@ bool fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_ } } } - loadAtPosition(in, memory, pos); - return true; + + saveUpToPosition(in, memory, pos); + return loadAtPosition(in, memory, pos); } void registerFileSegmentationEngineCSV(FormatFactory & factory) diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index ef5f3c0435c..409b24b496f 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -325,8 +325,9 @@ bool fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memor } } } - loadAtPosition(in, memory, pos) - return true; + + saveUpToPosition(in, memory, pos); + return loadAtPosition(in, memory, pos); } void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory) From fc055fc7a1ea201586025a5f201bd1e9aaa8cda2 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 22 Nov 2019 16:56:59 +0300 Subject: [PATCH 20/42] better documentation --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 4437ba79f7f..0ad80cfed2f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -999,7 +999,7 @@ Default value: 0. - Type: bool - Default value: True -Enable order-preserving parallel parsing of data formats. Supported only for TSV format. +Enable order-preserving parallel parsing of data formats. Supported only for TSV, TKSV, CSV and JSONEachRow formats. ## min_chunk_bytes_for_parallel_parsing From 55ee4800438f35959b91bb13be2cd2bbad0f4284 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 23 Nov 2019 03:12:50 +0300 Subject: [PATCH 21/42] Updated roadmap --- docs/ru/extended_roadmap.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/ru/extended_roadmap.md b/docs/ru/extended_roadmap.md index 5c89eb0284a..e0ca2261e28 100644 --- a/docs/ru/extended_roadmap.md +++ b/docs/ru/extended_roadmap.md @@ -571,6 +571,8 @@ Fuzzing тестирование - это тестирование случай ### 8.14. Запись данных в ORC. +Возможно, Андрей Коняев, ArenaData (зависит от желания). + ### 8.15. Запись данных в CapNProto. ### 8.16. Поддержка формата Avro. @@ -599,10 +601,16 @@ Fuzzing тестирование - это тестирование случай ### 8.20. Интеграция с SQS. +Низкий приоритет. + ### 8.21. Поддержка произвольного количества языков для имён регионов. Нужно для БК. Декабрь 2019. +### 8.22. Поддержка синтаксиса для переменных в стиле MySQL. + +При парсинге запроса преобразовывать синтаксис вида `@@version_full` в вызов функции `getGlobalVariable('version_full')`. Поддержать популярные MySQL переменные. Может быть поможет Юрий Баранов, если будет энтузиазм. + ## 9. Безопасность. From 0633f9f8db9ec54c65ae4766a68859ad6f2f896b Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 23 Nov 2019 03:24:25 +0300 Subject: [PATCH 22/42] Update CSVRowInputFormat.cpp --- dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 502504b4ffd..7564e760aa6 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -446,7 +446,7 @@ bool fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_ } else { - pos = find_first_symbols<'"','\r', '\n'>(pos, in.buffer().end()); + pos = find_first_symbols<'"', '\r', '\n'>(pos, in.buffer().end()); if (pos == in.buffer().end()) continue; if (*pos == '"') From d8d5697ce13a1f7f65e0ed5ebb83959149dc096f Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sat, 23 Nov 2019 04:33:27 +0300 Subject: [PATCH 23/42] Remove link to (almost) past meetup --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 32f6eba9bab..06ea8e94592 100644 --- a/README.md +++ b/README.md @@ -14,6 +14,5 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events -* [ClickHouse Meetup in Singapore](https://www.meetup.com/Singapore-Clickhouse-Meetup-Group/events/265085331/) on November 23. * [ClickHouse Meetup in San Francisco](https://www.eventbrite.com/e/clickhouse-december-meetup-registration-78642047481) on December 3. From 651e0f3a1c25f53f98b5e0e488a339a0c5fa9365 Mon Sep 17 00:00:00 2001 From: Pervakov Grigory Date: Thu, 14 Nov 2019 16:44:15 +0300 Subject: [PATCH 24/42] Update max_table_size_to_drop and max_partition_size_to_drop with config reload --- dbms/programs/server/Server.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index bd820e5eb0a..7952ccc9966 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -438,6 +438,13 @@ int Server::main(const std::vector & /*args*/) buildLoggers(*config, logger()); global_context->setClustersConfig(config); global_context->setMacros(std::make_unique(*config, "macros")); + + /// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default) + if (config->has("max_table_size_to_drop")) + global_context->setMaxTableSizeToDrop(config->getUInt64("max_table_size_to_drop")); + + if (config->has("max_partition_size_to_drop")) + global_context->setMaxPartitionSizeToDrop(config->getUInt64("max_partition_size_to_drop")); }, /* already_loaded = */ true); @@ -469,13 +476,6 @@ int Server::main(const std::vector & /*args*/) /// Limit on total number of concurrently executed queries. global_context->getProcessList().setMaxSize(config().getInt("max_concurrent_queries", 0)); - /// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default) - if (config().has("max_table_size_to_drop")) - global_context->setMaxTableSizeToDrop(config().getUInt64("max_table_size_to_drop")); - - if (config().has("max_partition_size_to_drop")) - global_context->setMaxPartitionSizeToDrop(config().getUInt64("max_partition_size_to_drop")); - /// Set up caches. /// Lower cache size on low-memory systems. From ac1a92649cb8e6c73dc48c2b266fd8f490b92a68 Mon Sep 17 00:00:00 2001 From: Pervakov Grigory Date: Fri, 15 Nov 2019 19:10:37 +0300 Subject: [PATCH 25/42] WIP: Add regression test on config reload --- .../__init__.py | 0 .../configs/config.xml | 31 +++++++++++ .../configs/users.xml | 23 ++++++++ .../test.py | 55 +++++++++++++++++++ 4 files changed, 109 insertions(+) create mode 100644 dbms/tests/integration/test_reload_max_table_size_to_drop/__init__.py create mode 100644 dbms/tests/integration/test_reload_max_table_size_to_drop/configs/config.xml create mode 100644 dbms/tests/integration/test_reload_max_table_size_to_drop/configs/users.xml create mode 100644 dbms/tests/integration/test_reload_max_table_size_to_drop/test.py diff --git a/dbms/tests/integration/test_reload_max_table_size_to_drop/__init__.py b/dbms/tests/integration/test_reload_max_table_size_to_drop/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_reload_max_table_size_to_drop/configs/config.xml b/dbms/tests/integration/test_reload_max_table_size_to_drop/configs/config.xml new file mode 100644 index 00000000000..b5e5495c096 --- /dev/null +++ b/dbms/tests/integration/test_reload_max_table_size_to_drop/configs/config.xml @@ -0,0 +1,31 @@ + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + + 9000 + 127.0.0.1 + + + + true + none + + AcceptCertificateHandler + + + + + 500 + 5368709120 + ./clickhouse/ + users.xml + + 1 + 1 + diff --git a/dbms/tests/integration/test_reload_max_table_size_to_drop/configs/users.xml b/dbms/tests/integration/test_reload_max_table_size_to_drop/configs/users.xml new file mode 100644 index 00000000000..6061af8e33d --- /dev/null +++ b/dbms/tests/integration/test_reload_max_table_size_to_drop/configs/users.xml @@ -0,0 +1,23 @@ + + + + + + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/dbms/tests/integration/test_reload_max_table_size_to_drop/test.py b/dbms/tests/integration/test_reload_max_table_size_to_drop/test.py new file mode 100644 index 00000000000..46309ed0cf3 --- /dev/null +++ b/dbms/tests/integration/test_reload_max_table_size_to_drop/test.py @@ -0,0 +1,55 @@ +import time +import pytest +import sys +import os + +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance('node', config_dir="configs") + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +config_dir = os.path.join(SCRIPT_DIR, './_instances/node/configs') + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + node.query("CREATE DATABASE chtest;") + node.query( + ''' + CREATE TABLE chtest.reload_max_table_size (date Date, id UInt32) + ENGINE = MergeTree(date, id, 8192) + ''' + ) + node.query("INSERT INTO chtest.reload_max_table_size VALUES (now(), 0)") + yield cluster + finally: + #cluster.shutdown() + pass + + +def test_reload_max_table_size_to_drop(start_cluster): + config = open(config_dir + '/config.xml', 'r') + config_lines = config.readlines() + config.close() + + error = node.get_query_request("DROP TABLE chtest.reload_max_table_size") # change to query_and_get_error after fix + print >> sys.stderr, 'error: ' + error.get_answer() + print >> sys.stderr, 'path: ' + config_dir + print >> sys.stderr, 'config: ' + for line in config_lines: + print >> sys.stderr, line + + assert error != "" # Crashes due to illigal config + + config_lines = map(lambda line: line.replace("1", "1000000"), + config_lines) + config = open(config_dir + '/config.xml', 'w') + config.writelines(config_lines) + + time.sleep(50000) + error = node.query_and_get_error("DROP TABLE chtest.reload_max_table_size") + assert error == "" From 4bebf34215236c6be33057f7489ba5ec83b326f9 Mon Sep 17 00:00:00 2001 From: Pervakov Grigory Date: Mon, 18 Nov 2019 18:33:43 +0300 Subject: [PATCH 26/42] Complete test with sleep --- .../test.py | 48 ++++++++----------- 1 file changed, 21 insertions(+), 27 deletions(-) diff --git a/dbms/tests/integration/test_reload_max_table_size_to_drop/test.py b/dbms/tests/integration/test_reload_max_table_size_to_drop/test.py index 46309ed0cf3..9c38a879eb4 100644 --- a/dbms/tests/integration/test_reload_max_table_size_to_drop/test.py +++ b/dbms/tests/integration/test_reload_max_table_size_to_drop/test.py @@ -1,6 +1,5 @@ import time import pytest -import sys import os from helpers.cluster import ClickHouseCluster @@ -10,46 +9,41 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', config_dir="configs") SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -config_dir = os.path.join(SCRIPT_DIR, './_instances/node/configs') +CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/node/configs/config.xml') @pytest.fixture(scope="module") def start_cluster(): try: cluster.start() - node.query("CREATE DATABASE chtest;") - node.query( - ''' - CREATE TABLE chtest.reload_max_table_size (date Date, id UInt32) - ENGINE = MergeTree(date, id, 8192) - ''' - ) - node.query("INSERT INTO chtest.reload_max_table_size VALUES (now(), 0)") + node.query("CREATE TABLE test(date Date, id UInt32) ENGINE = MergeTree(date, id, 8192)") yield cluster finally: - #cluster.shutdown() - pass + cluster.shutdown() def test_reload_max_table_size_to_drop(start_cluster): - config = open(config_dir + '/config.xml', 'r') + node.query("INSERT INTO test VALUES (now(), 0)") + + time.sleep(5) # wait for data part commit + + drop = node.get_query_request("DROP TABLE test") + out, err = drop.get_answer_and_error() + assert out == "" + assert err != "" + + config = open(CONFIG_PATH, 'r') config_lines = config.readlines() config.close() - - error = node.get_query_request("DROP TABLE chtest.reload_max_table_size") # change to query_and_get_error after fix - print >> sys.stderr, 'error: ' + error.get_answer() - print >> sys.stderr, 'path: ' + config_dir - print >> sys.stderr, 'config: ' - for line in config_lines: - print >> sys.stderr, line - - assert error != "" # Crashes due to illigal config - config_lines = map(lambda line: line.replace("1", "1000000"), config_lines) - config = open(config_dir + '/config.xml', 'w') + config = open(CONFIG_PATH, 'w') config.writelines(config_lines) + config.close() - time.sleep(50000) - error = node.query_and_get_error("DROP TABLE chtest.reload_max_table_size") - assert error == "" + time.sleep(5) # wait for config reload + + drop = node.get_query_request("DROP TABLE test") + out, err = drop.get_answer_and_error() + assert out == "" + assert err == "" From 1f1e25dcebb5aa5dd68c52c14415d390a17cb719 Mon Sep 17 00:00:00 2001 From: Grigory Pervakov Date: Tue, 19 Nov 2019 18:37:58 +0300 Subject: [PATCH 27/42] Update dbms/tests/integration/test_reload_max_table_size_to_drop/test.py Co-Authored-By: Alexander Burmak --- .../integration/test_reload_max_table_size_to_drop/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/integration/test_reload_max_table_size_to_drop/test.py b/dbms/tests/integration/test_reload_max_table_size_to_drop/test.py index 9c38a879eb4..dd506259ef4 100644 --- a/dbms/tests/integration/test_reload_max_table_size_to_drop/test.py +++ b/dbms/tests/integration/test_reload_max_table_size_to_drop/test.py @@ -41,7 +41,7 @@ def test_reload_max_table_size_to_drop(start_cluster): config.writelines(config_lines) config.close() - time.sleep(5) # wait for config reload + node.query("SYSTEM RELOAD CONFIG") drop = node.get_query_request("DROP TABLE test") out, err = drop.get_answer_and_error() From 64cb07918688235339b3292defd889414672e2df Mon Sep 17 00:00:00 2001 From: Grigory Pervakov Date: Tue, 19 Nov 2019 18:38:11 +0300 Subject: [PATCH 28/42] Update dbms/tests/integration/test_reload_max_table_size_to_drop/test.py Co-Authored-By: Alexander Burmak --- .../integration/test_reload_max_table_size_to_drop/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/integration/test_reload_max_table_size_to_drop/test.py b/dbms/tests/integration/test_reload_max_table_size_to_drop/test.py index dd506259ef4..3959b383fc5 100644 --- a/dbms/tests/integration/test_reload_max_table_size_to_drop/test.py +++ b/dbms/tests/integration/test_reload_max_table_size_to_drop/test.py @@ -16,7 +16,7 @@ CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/node/configs/config.xml') def start_cluster(): try: cluster.start() - node.query("CREATE TABLE test(date Date, id UInt32) ENGINE = MergeTree(date, id, 8192)") + node.query("CREATE TABLE test(date Date, id UInt32) ENGINE = MergeTree() PARTITION BY date ORDER BY id") yield cluster finally: cluster.shutdown() From 9fbc55c61aa9faaa273bc531dfa42813888c62c1 Mon Sep 17 00:00:00 2001 From: Pervakov Grigory Date: Wed, 20 Nov 2019 19:40:27 +0300 Subject: [PATCH 29/42] Change max_table_size_to_drop and max_partition_size_to_drop to atmoic. Update hint in default config. --- dbms/programs/server/config.xml | 2 +- dbms/src/Interpreters/Context.cpp | 16 ++++++++-------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index 6e9bb527c97..248d37bc39d 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -411,7 +411,7 @@