Merge remote-tracking branch 'remotes/clickhouse/master' into sevirov-DOCSUP-9738-types_supported_in_dictionaries

This commit is contained in:
Dmitriy 2021-06-04 00:34:58 +03:00
commit fbcd22ab4e
221 changed files with 6501 additions and 2999 deletions

2
contrib/NuRaft vendored

@ -1 +1 @@
Subproject commit 95d6bbba579b3a4e4c2dede954f541ff6f3dba51
Subproject commit 2a1bf7d87b4a03561fc66fbb49cee8a288983c5d

View File

@ -1,5 +1,5 @@
# docker build -t yandex/clickhouse-integration-tests-runner .
FROM ubuntu:18.04
FROM ubuntu:20.04
RUN apt-get update \
&& env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
@ -14,7 +14,6 @@ RUN apt-get update \
wget \
git \
iproute2 \
module-init-tools \
cgroupfs-mount \
python3-pip \
tzdata \
@ -42,7 +41,6 @@ ENV TZ=Europe/Moscow
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
ENV DOCKER_CHANNEL stable
ENV DOCKER_VERSION 5:19.03.13~3-0~ubuntu-bionic
RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add -
RUN add-apt-repository "deb [arch=amd64] https://download.docker.com/linux/ubuntu $(lsb_release -c -s) ${DOCKER_CHANNEL}"
@ -66,17 +64,18 @@ RUN python3 -m pip install \
dict2xml \
dicttoxml \
docker \
docker-compose==1.22.0 \
docker-compose==1.28.2 \
grpcio \
grpcio-tools \
kafka-python \
kazoo \
minio \
protobuf \
psycopg2-binary==2.7.5 \
psycopg2-binary==2.8.6 \
pymongo \
pytest \
pytest-timeout \
pytest-xdist \
redis \
tzlocal \
urllib3 \
@ -86,6 +85,7 @@ RUN python3 -m pip install \
COPY modprobe.sh /usr/local/bin/modprobe
COPY dockerd-entrypoint.sh /usr/local/bin/
COPY compose/ /compose/
COPY misc/ /misc/
RUN set -x \
&& addgroup --system dockremap \
@ -94,7 +94,6 @@ RUN set -x \
&& echo 'dockremap:165536:65536' >> /etc/subuid \
&& echo 'dockremap:165536:65536' >> /etc/subgid
VOLUME /var/lib/docker
EXPOSE 2375
ENTRYPOINT ["dockerd-entrypoint.sh"]
CMD ["sh", "-c", "pytest $PYTEST_OPTS"]

View File

@ -1,7 +1,5 @@
version: '2.3'
services:
cassandra1:
image: cassandra
image: cassandra:4.0
restart: always
ports:
- 9043:9042

View File

@ -5,6 +5,10 @@ services:
hostname: hdfs1
restart: always
ports:
- 50075:50075
- 50070:50070
- ${HDFS_NAME_EXTERNAL_PORT}:${HDFS_NAME_INTERNAL_PORT} #50070
- ${HDFS_DATA_EXTERNAL_PORT}:${HDFS_DATA_INTERNAL_PORT} #50075
entrypoint: /etc/bootstrap.sh -d
volumes:
- type: ${HDFS_FS:-tmpfs}
source: ${HDFS_LOGS:-}
target: /usr/local/hadoop/logs

View File

@ -15,10 +15,11 @@ services:
image: confluentinc/cp-kafka:5.2.0
hostname: kafka1
ports:
- "9092:9092"
- ${KAFKA_EXTERNAL_PORT}:${KAFKA_EXTERNAL_PORT}
environment:
KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kafka1:19092
KAFKA_LISTENERS: INSIDE://:9092,OUTSIDE://:19092
KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA_EXTERNAL_PORT},OUTSIDE://kafka1:19092
KAFKA_ADVERTISED_HOST_NAME: kafka1
KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19092
KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT
KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE
KAFKA_BROKER_ID: 1
@ -34,7 +35,7 @@ services:
image: confluentinc/cp-schema-registry:5.2.0
hostname: schema-registry
ports:
- "8081:8081"
- ${SCHEMA_REGISTRY_EXTERNAL_PORT}:${SCHEMA_REGISTRY_INTERNAL_PORT}
environment:
SCHEMA_REGISTRY_HOST_NAME: schema-registry
SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT

View File

@ -11,10 +11,12 @@ services:
- ${KERBERIZED_HDFS_DIR}/../../hdfs_configs/bootstrap.sh:/etc/bootstrap.sh:ro
- ${KERBERIZED_HDFS_DIR}/secrets:/usr/local/hadoop/etc/hadoop/conf
- ${KERBERIZED_HDFS_DIR}/secrets/krb_long.conf:/etc/krb5.conf:ro
- type: ${KERBERIZED_HDFS_FS:-tmpfs}
source: ${KERBERIZED_HDFS_LOGS:-}
target: /var/log/hadoop-hdfs
ports:
- 1006:1006
- 50070:50070
- 9010:9010
- ${KERBERIZED_HDFS_NAME_EXTERNAL_PORT}:${KERBERIZED_HDFS_NAME_INTERNAL_PORT} #50070
- ${KERBERIZED_HDFS_DATA_EXTERNAL_PORT}:${KERBERIZED_HDFS_DATA_INTERNAL_PORT} #1006
depends_on:
- hdfskerberos
entrypoint: /etc/bootstrap.sh -d

View File

@ -23,13 +23,13 @@ services:
# restart: always
hostname: kerberized_kafka1
ports:
- "9092:9092"
- "9093:9093"
- ${KERBERIZED_KAFKA_EXTERNAL_PORT}:${KERBERIZED_KAFKA_EXTERNAL_PORT}
environment:
KAFKA_LISTENERS: OUTSIDE://:19092,UNSECURED_OUTSIDE://:19093,UNSECURED_INSIDE://:9093
KAFKA_ADVERTISED_LISTENERS: OUTSIDE://kerberized_kafka1:19092,UNSECURED_OUTSIDE://kerberized_kafka1:19093,UNSECURED_INSIDE://localhost:9093
KAFKA_LISTENERS: OUTSIDE://:19092,UNSECURED_OUTSIDE://:19093,UNSECURED_INSIDE://0.0.0.0:${KERBERIZED_KAFKA_EXTERNAL_PORT}
KAFKA_ADVERTISED_LISTENERS: OUTSIDE://kerberized_kafka1:19092,UNSECURED_OUTSIDE://kerberized_kafka1:19093,UNSECURED_INSIDE://localhost:${KERBERIZED_KAFKA_EXTERNAL_PORT}
# KAFKA_LISTENERS: INSIDE://kerberized_kafka1:9092,OUTSIDE://kerberized_kafka1:19092
# KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kerberized_kafka1:19092
KAFKA_ADVERTISED_HOST_NAME: kerberized_kafka1
KAFKA_SASL_MECHANISM_INTER_BROKER_PROTOCOL: GSSAPI
KAFKA_SASL_ENABLED_MECHANISMS: GSSAPI
KAFKA_SASL_KERBEROS_SERVICE_NAME: kafka

View File

@ -6,8 +6,8 @@ services:
volumes:
- data1-1:/data1
- ${MINIO_CERTS_DIR:-}:/certs
ports:
- "9001:9001"
expose:
- ${MINIO_PORT}
environment:
MINIO_ACCESS_KEY: minio
MINIO_SECRET_KEY: minio123
@ -20,14 +20,14 @@ services:
# HTTP proxies for Minio.
proxy1:
image: yandex/clickhouse-s3-proxy
ports:
expose:
- "8080" # Redirect proxy port
- "80" # Reverse proxy port
- "443" # Reverse proxy port (secure)
proxy2:
image: yandex/clickhouse-s3-proxy
ports:
expose:
- "8080"
- "80"
- "443"
@ -35,7 +35,7 @@ services:
# Empty container to run proxy resolver.
resolver:
image: yandex/clickhouse-python-bottle
ports:
expose:
- "8080"
tty: true
depends_on:

View File

@ -7,5 +7,5 @@ services:
MONGO_INITDB_ROOT_USERNAME: root
MONGO_INITDB_ROOT_PASSWORD: clickhouse
ports:
- 27018:27017
- ${MONGO_EXTERNAL_PORT}:${MONGO_INTERNAL_PORT}
command: --profile=2 --verbose

View File

@ -1,10 +1,24 @@
version: '2.3'
services:
mysql1:
mysql57:
image: mysql:5.7
restart: always
environment:
MYSQL_ROOT_PASSWORD: clickhouse
ports:
- 3308:3306
command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency
MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST}
DATADIR: /mysql/
expose:
- ${MYSQL_PORT}
command: --server_id=100
--log-bin='mysql-bin-1.log'
--default-time-zone='+3:00'
--gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3
--log-error=/mysql/error.log
--general-log=ON
--general-log-file=/mysql/general.log
volumes:
- type: ${MYSQL_LOGS_FS:-tmpfs}
source: ${MYSQL_LOGS:-}
target: /mysql/

View File

@ -12,3 +12,10 @@ services:
--gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3
--log-error=/var/log/mysqld/error.log
--general-log=ON
--general-log-file=/var/log/mysqld/general.log
volumes:
- type: ${MYSQL_LOGS_FS:-tmpfs}
source: ${MYSQL_LOGS:-}
target: /var/log/mysqld/

View File

@ -0,0 +1,23 @@
version: '2.3'
services:
mysql80:
image: mysql:8.0
restart: always
environment:
MYSQL_ROOT_PASSWORD: clickhouse
MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST}
DATADIR: /mysql/
expose:
- ${MYSQL8_PORT}
command: --server_id=100 --log-bin='mysql-bin-1.log'
--default_authentication_plugin='mysql_native_password'
--default-time-zone='+3:00' --gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3
--log-error=/mysql/error.log
--general-log=ON
--general-log-file=/mysql/general.log
volumes:
- type: ${MYSQL8_LOGS_FS:-tmpfs}
source: ${MYSQL8_LOGS:-}
target: /mysql/

View File

@ -1,15 +0,0 @@
version: '2.3'
services:
mysql8_0:
image: mysql:8.0
restart: 'no'
environment:
MYSQL_ROOT_PASSWORD: clickhouse
ports:
- 3309:3306
command: --server_id=100 --log-bin='mysql-bin-1.log'
--default_authentication_plugin='mysql_native_password'
--default-time-zone='+3:00'
--gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3

View File

@ -1,6 +1,6 @@
version: '2.3'
services:
mysql1:
mysql_client:
image: mysql:5.7
restart: always
environment:

View File

@ -5,19 +5,64 @@ services:
restart: always
environment:
MYSQL_ROOT_PASSWORD: clickhouse
ports:
- 3348:3306
MYSQL_ROOT_HOST: ${MYSQL_CLUSTER_ROOT_HOST}
DATADIR: /mysql/
expose:
- ${MYSQL_CLUSTER_PORT}
command: --server_id=100
--log-bin='mysql-bin-2.log'
--default-time-zone='+3:00'
--gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3
--log-error=/mysql/2_error.log
--general-log=ON
--general-log-file=/mysql/2_general.log
volumes:
- type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs}
source: ${MYSQL_CLUSTER_LOGS:-}
target: /mysql/
mysql3:
image: mysql:5.7
restart: always
environment:
MYSQL_ROOT_PASSWORD: clickhouse
ports:
- 3388:3306
MYSQL_ROOT_HOST: ${MYSQL_CLUSTER_ROOT_HOST}
DATADIR: /mysql/
expose:
- ${MYSQL_CLUSTER_PORT}
command: --server_id=100
--log-bin='mysql-bin-3.log'
--default-time-zone='+3:00'
--gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3
--log-error=/mysql/3_error.log
--general-log=ON
--general-log-file=/mysql/3_general.log
volumes:
- type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs}
source: ${MYSQL_CLUSTER_LOGS:-}
target: /mysql/
mysql4:
image: mysql:5.7
restart: always
environment:
MYSQL_ROOT_PASSWORD: clickhouse
ports:
- 3368:3306
MYSQL_ROOT_HOST: ${MYSQL_CLUSTER_ROOT_HOST}
DATADIR: /mysql/
expose:
- ${MYSQL_CLUSTER_PORT}
command: --server_id=100
--log-bin='mysql-bin-4.log'
--default-time-zone='+3:00'
--gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3
--log-error=/mysql/4_error.log
--general-log=ON
--general-log-file=/mysql/4_general.log
volumes:
- type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs}
source: ${MYSQL_CLUSTER_LOGS:-}
target: /mysql/

View File

@ -2,12 +2,24 @@ version: '2.3'
services:
postgres1:
image: postgres
command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"]
restart: always
environment:
POSTGRES_PASSWORD: mysecretpassword
ports:
- 5432:5432
expose:
- ${POSTGRES_PORT}
healthcheck:
test: ["CMD-SHELL", "pg_isready -U postgres"]
interval: 10s
timeout: 5s
retries: 5
networks:
default:
aliases:
- postgre-sql.local
default:
aliases:
- postgre-sql.local
environment:
POSTGRES_HOST_AUTH_METHOD: "trust"
POSTGRES_PASSWORD: mysecretpassword
PGDATA: /postgres/data
volumes:
- type: ${POSTGRES_LOGS_FS:-tmpfs}
source: ${POSTGRES_DIR:-}
target: /postgres/

View File

@ -2,22 +2,43 @@ version: '2.3'
services:
postgres2:
image: postgres
command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"]
restart: always
environment:
POSTGRES_HOST_AUTH_METHOD: "trust"
POSTGRES_PASSWORD: mysecretpassword
ports:
- 5421:5432
PGDATA: /postgres/data
expose:
- ${POSTGRES_PORT}
volumes:
- type: ${POSTGRES_LOGS_FS:-tmpfs}
source: ${POSTGRES2_DIR:-}
target: /postgres/
postgres3:
image: postgres
command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"]
restart: always
environment:
POSTGRES_HOST_AUTH_METHOD: "trust"
POSTGRES_PASSWORD: mysecretpassword
ports:
- 5441:5432
PGDATA: /postgres/data
expose:
- ${POSTGRES_PORT}
volumes:
- type: ${POSTGRES_LOGS_FS:-tmpfs}
source: ${POSTGRES3_DIR:-}
target: /postgres/
postgres4:
image: postgres
command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"]
restart: always
environment:
POSTGRES_HOST_AUTH_METHOD: "trust"
POSTGRES_PASSWORD: mysecretpassword
ports:
- 5461:5432
PGDATA: /postgres/data
expose:
- ${POSTGRES_PORT}
volumes:
- type: ${POSTGRES_LOGS_FS:-tmpfs}
source: ${POSTGRES4_DIR:-}
target: /postgres/

View File

@ -2,11 +2,15 @@ version: '2.3'
services:
rabbitmq1:
image: rabbitmq:3-management
image: rabbitmq:3-management-alpine
hostname: rabbitmq1
ports:
- "5672:5672"
- "15672:15672"
expose:
- ${RABBITMQ_PORT}
environment:
RABBITMQ_DEFAULT_USER: "root"
RABBITMQ_DEFAULT_PASS: "clickhouse"
RABBITMQ_LOG_BASE: /rabbitmq_logs/
volumes:
- type: ${RABBITMQ_LOGS_FS:-tmpfs}
source: ${RABBITMQ_LOGS:-}
target: /rabbitmq_logs/

View File

@ -4,5 +4,5 @@ services:
image: redis
restart: always
ports:
- 6380:6379
- ${REDIS_EXTERNAL_PORT}:${REDIS_INTERNAL_PORT}
command: redis-server --requirepass "clickhouse" --databases 32

View File

@ -0,0 +1,75 @@
version: '2.3'
services:
zoo1:
image: zookeeper:3.6.2
restart: always
environment:
ZOO_TICK_TIME: 500
ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888;2181
ZOO_MY_ID: 1
JVMFLAGS: -Dzookeeper.forceSync=no
ZOO_SECURE_CLIENT_PORT: $ZOO_SECURE_CLIENT_PORT
command: ["zkServer.sh", "start-foreground"]
entrypoint: /zookeeper-ssl-entrypoint.sh
volumes:
- type: bind
source: /misc/zookeeper-ssl-entrypoint.sh
target: /zookeeper-ssl-entrypoint.sh
- type: bind
source: /misc/client.crt
target: /clickhouse-config/client.crt
- type: ${ZK_FS:-tmpfs}
source: ${ZK_DATA1:-}
target: /data
- type: ${ZK_FS:-tmpfs}
source: ${ZK_DATA_LOG1:-}
target: /datalog
zoo2:
image: zookeeper:3.6.2
restart: always
environment:
ZOO_TICK_TIME: 500
ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888
ZOO_MY_ID: 2
JVMFLAGS: -Dzookeeper.forceSync=no
ZOO_SECURE_CLIENT_PORT: $ZOO_SECURE_CLIENT_PORT
command: ["zkServer.sh", "start-foreground"]
entrypoint: /zookeeper-ssl-entrypoint.sh
volumes:
- type: bind
source: /misc/zookeeper-ssl-entrypoint.sh
target: /zookeeper-ssl-entrypoint.sh
- type: bind
source: /misc/client.crt
target: /clickhouse-config/client.crt
- type: ${ZK_FS:-tmpfs}
source: ${ZK_DATA2:-}
target: /data
- type: ${ZK_FS:-tmpfs}
source: ${ZK_DATA_LOG2:-}
target: /datalog
zoo3:
image: zookeeper:3.6.2
restart: always
environment:
ZOO_TICK_TIME: 500
ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888;2181
ZOO_MY_ID: 3
JVMFLAGS: -Dzookeeper.forceSync=no
ZOO_SECURE_CLIENT_PORT: $ZOO_SECURE_CLIENT_PORT
command: ["zkServer.sh", "start-foreground"]
entrypoint: /zookeeper-ssl-entrypoint.sh
volumes:
- type: bind
source: /misc/zookeeper-ssl-entrypoint.sh
target: /zookeeper-ssl-entrypoint.sh
- type: bind
source: /misc/client.crt
target: /clickhouse-config/client.crt
- type: ${ZK_FS:-tmpfs}
source: ${ZK_DATA3:-}
target: /data
- type: ${ZK_FS:-tmpfs}
source: ${ZK_DATA_LOG3:-}
target: /datalog

View File

@ -2,17 +2,17 @@
set -e
mkdir -p /etc/docker/
cat > /etc/docker/daemon.json << EOF
{
echo '{
"ipv6": true,
"fixed-cidr-v6": "fd00::/8",
"ip-forward": true,
"log-level": "debug",
"storage-driver": "overlay2",
"insecure-registries" : ["dockerhub-proxy.sas.yp-c.yandex.net:5000"],
"registry-mirrors" : ["http://dockerhub-proxy.sas.yp-c.yandex.net:5000"]
}
EOF
}' | dd of=/etc/docker/daemon.json
dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 &>/var/log/somefile &
dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log &
set +e
reties=0

View File

@ -0,0 +1,19 @@
-----BEGIN CERTIFICATE-----
MIIC/TCCAeWgAwIBAgIJANjx1QSR77HBMA0GCSqGSIb3DQEBCwUAMBQxEjAQBgNV
BAMMCWxvY2FsaG9zdDAgFw0xODA3MzAxODE2MDhaGA8yMjkyMDUxNDE4MTYwOFow
FDESMBAGA1UEAwwJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIB
CgKCAQEAs9uSo6lJG8o8pw0fbVGVu0tPOljSWcVSXH9uiJBwlZLQnhN4SFSFohfI
4K8U1tBDTnxPLUo/V1K9yzoLiRDGMkwVj6+4+hE2udS2ePTQv5oaMeJ9wrs+5c9T
4pOtlq3pLAdm04ZMB1nbrEysceVudHRkQbGHzHp6VG29Fw7Ga6YpqyHQihRmEkTU
7UCYNA+Vk7aDPdMS/khweyTpXYZimaK9f0ECU3/VOeG3fH6Sp2X6FN4tUj/aFXEj
sRmU5G2TlYiSIUMF2JPdhSihfk1hJVALrHPTU38SOL+GyyBRWdNcrIwVwbpvsvPg
pryMSNxnpr0AK0dFhjwnupIv5hJIOQIDAQABo1AwTjAdBgNVHQ4EFgQUjPLb3uYC
kcamyZHK4/EV8jAP0wQwHwYDVR0jBBgwFoAUjPLb3uYCkcamyZHK4/EV8jAP0wQw
DAYDVR0TBAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEAM/ocuDvfPus/KpMVD51j
4IdlU8R0vmnYLQ+ygzOAo7+hUWP5j0yvq4ILWNmQX6HNvUggCgFv9bjwDFhb/5Vr
85ieWfTd9+LTjrOzTw4avdGwpX9G+6jJJSSq15tw5ElOIFb/qNA9O4dBiu8vn03C
L/zRSXrARhSqTW5w/tZkUcSTT+M5h28+Lgn9ysx4Ff5vi44LJ1NnrbJbEAIYsAAD
+UA+4MBFKx1r6hHINULev8+lCfkpwIaeS8RL+op4fr6kQPxnULw8wT8gkuc8I4+L
P9gg/xDHB44T3ADGZ5Ib6O0DJaNiToO6rnoaaxs0KkotbvDWvRoxEytSbXKoYjYp
0g==
-----END CERTIFICATE-----

View File

@ -81,8 +81,8 @@ if [[ ! -f "$ZOO_DATA_DIR/myid" ]]; then
echo "${ZOO_MY_ID:-1}" > "$ZOO_DATA_DIR/myid"
fi
mkdir -p $(dirname $ZOO_SSL_KEYSTORE_LOCATION)
mkdir -p $(dirname $ZOO_SSL_TRUSTSTORE_LOCATION)
mkdir -p "$(dirname $ZOO_SSL_KEYSTORE_LOCATION)"
mkdir -p "$(dirname $ZOO_SSL_TRUSTSTORE_LOCATION)"
if [[ ! -f "$ZOO_SSL_KEYSTORE_LOCATION" ]]; then
keytool -genkeypair -alias zookeeper -keyalg RSA -validity 365 -keysize 2048 -dname "cn=zookeeper" -keypass password -keystore $ZOO_SSL_KEYSTORE_LOCATION -storepass password -deststoretype pkcs12

View File

@ -3,7 +3,6 @@ FROM ubuntu:20.04
RUN apt-get update --yes && env DEBIAN_FRONTEND=noninteractive apt-get install wget unzip git openjdk-14-jdk maven python3 --yes --no-install-recommends
RUN wget https://github.com/sqlancer/sqlancer/archive/master.zip -O /sqlancer.zip
RUN mkdir /sqlancer && \
cd /sqlancer && \
unzip /sqlancer.zip

View File

@ -90,7 +90,7 @@ clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits"
clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits"
clickhouse-client --query "SHOW TABLES FROM test"
./stress --hung-check --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" \
./stress --hung-check --drop-databases --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" \
&& echo -e 'Test script exit code\tOK' >> /test_output/test_results.tsv \
|| echo -e 'Test script failed\tFAIL' >> /test_output/test_results.tsv

View File

@ -19,25 +19,25 @@ def get_skip_list_cmd(path):
def get_options(i):
options = ""
options = []
if 0 < i:
options += " --order=random"
options.append("--order=random")
if i % 3 == 1:
options += " --db-engine=Ordinary"
options.append("--db-engine=Ordinary")
if i % 3 == 2:
options += ''' --client-option='allow_experimental_database_replicated=1' --db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i)
options.append('''--client-option='allow_experimental_database_replicated=1' --db-engine="Replicated('/test/db/test_{}', 's1', 'r1')"'''.format(i))
# If database name is not specified, new database is created for each functional test.
# Run some threads with one database for all tests.
if i % 2 == 1:
options += " --database=test_{}".format(i)
options.append(" --database=test_{}".format(i))
if i == 13:
options += " --client-option='memory_tracker_fault_probability=0.00001'"
options.append(" --client-option='memory_tracker_fault_probability=0.00001'")
return options
return ' '.join(options)
def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_time_limit):
@ -62,7 +62,7 @@ def compress_stress_logs(output_path, files_prefix):
cmd = f"cd {output_path} && tar -zcf stress_run_logs.tar.gz {files_prefix}* && rm {files_prefix}*"
check_output(cmd, shell=True)
def prepare_for_hung_check():
def prepare_for_hung_check(drop_databases):
# FIXME this function should not exist, but...
# We attach gdb to clickhouse-server before running tests
@ -95,6 +95,17 @@ def prepare_for_hung_check():
# Long query from 00084_external_agregation
call("""clickhouse client -q "KILL QUERY WHERE query LIKE 'SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u %'" """, shell=True, stderr=STDOUT)
if drop_databases:
# Here we try to drop all databases in async mode. If some queries really hung, than drop will hung too.
# Otherwise we will get rid of queries which wait for background pool. It can take a long time on slow builds (more than 900 seconds).
databases = check_output('clickhouse client -q "SHOW DATABASES"', shell=True).decode('utf-8').strip().split()
for db in databases:
if db == "system":
continue
command = f'clickhouse client -q "DROP DATABASE {db}"'
# we don't wait for drop
Popen(command, shell=True)
# Wait for last queries to finish if any, not longer than 300 seconds
call("""clickhouse client -q "select sleepEachRow((
select maxOrDefault(300 - elapsed) + 1 from system.processes where query not like '%from system.processes%' and elapsed < 300
@ -122,8 +133,12 @@ if __name__ == "__main__":
parser.add_argument("--global-time-limit", type=int, default=3600)
parser.add_argument("--num-parallel", type=int, default=cpu_count())
parser.add_argument('--hung-check', action='store_true', default=False)
# make sense only for hung check
parser.add_argument('--drop-databases', action='store_true', default=False)
args = parser.parse_args()
if args.drop_databases and not args.hung_check:
raise Exception("--drop-databases only used in hung check (--hung-check)")
func_pipes = []
func_pipes = run_func_test(args.test_cmd, args.output_folder, args.num_parallel, args.skip_func_tests, args.global_time_limit)
@ -145,7 +160,7 @@ if __name__ == "__main__":
logging.info("Logs compressed")
if args.hung_check:
have_long_running_queries = prepare_for_hung_check()
have_long_running_queries = prepare_for_hung_check(args.drop_databases)
logging.info("Checking if some queries hung")
cmd = "{} {} {}".format(args.test_cmd, "--hung-check", "00001_select_1")
res = call(cmd, shell=True, stderr=STDOUT)

View File

@ -123,6 +123,19 @@ The `Insert` command creates one or more blocks (parts). When inserting into Rep
A large number of `replicated_deduplication_window` slows down `Inserts` because it needs to compare more entries.
The hash sum is calculated from the composition of the field names and types and the data of the inserted part (stream of bytes).
## non_replicated_deduplication_window {#non-replicated-deduplication-window}
The number of the most recently inserted blocks in the non-replicated [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table for which hash sums are stored to check for duplicates.
Possible values:
- Any positive integer.
- 0 (disable deduplication).
Default value: 0.
A deduplication mechanism is used, similar to replicated tables (see [replicated_deduplication_window](#replicated-deduplication-window) setting). The hash sums of the created parts are written to a local file on a disk.
## replicated_deduplication_window_seconds {#replicated-deduplication-window-seconds}
The number of seconds after which the hash sums of the inserted blocks are removed from Zookeeper.

View File

@ -506,3 +506,256 @@ Solution: Write in the GROUP BY query SearchPhrase HAVING uniqUpTo(4)(UserID) >=
## sumMapFiltered(keys_to_keep)(keys, values) {#summapfilteredkeys-to-keepkeys-values}
Same behavior as [sumMap](../../sql-reference/aggregate-functions/reference/summap.md#agg_functions-summap) except that an array of keys is passed as a parameter. This can be especially useful when working with a high cardinality of keys.
## sequenceNextNode {#sequenceNextNode}
Returns a value of next event that matched an event chain.
_Experimental function, `SET allow_experimental_funnel_functions = 1` to enable it._
**Syntax**
``` sql
sequenceNextNode(direction, base)(timestamp, event_column, base_condition, event1, event2, event3, ...)
```
**Parameters**
- `direction` - Used to navigate to directions.
- forward : Moving forward
- backward: Moving backward
- `base` - Used to set the base point.
- head : Set the base point to the first event
- tail : Set the base point to the last event
- first_match : Set the base point to the first matched event1
- last_match : Set the base point to the last matched event1
**Arguments**
- `timestamp` — Name of the column containing the timestamp. Data types supported: `Date`, `DateTime` and other unsigned integer types.
- `event_column` — Name of the column containing the value of the next event to be returned. Data types supported: `String` and `Nullable(String)`
- `base_condition` — Condition that the base point must fulfill.
- `cond` — Conditions describing the chain of events. `UInt8`
**Returned value**
- `event_column[next_index]` - if the pattern is matched and next value exists.
- `NULL` - if the pattern isnt matched or next value doesn't exist.
Type: `Nullable(String)`.
**Example**
It can be used when events are A->B->C->E->F and you want to know the event following B->C, which is E.
The query statement searching the event following A->B :
``` sql
CREATE TABLE test_flow (
dt DateTime,
id int,
page String)
ENGINE = MergeTree()
PARTITION BY toYYYYMMDD(dt)
ORDER BY id;
INSERT INTO test_flow VALUES (1, 1, 'A') (2, 1, 'B') (3, 1, 'C') (4, 1, 'E') (5, 1, 'F');
SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'A', page = 'A', page = 'B') as next_flow FROM test_flow GROUP BY id;
```
Result:
``` text
┌─id─┬─next_flow─┐
│ 1 │ C │
└────┴───────────┘
```
**Behavior for `forward` and `head`**
```SQL
ALTER TABLE test_flow DELETE WHERE 1 = 1 settings mutations_sync = 1;
INSERT INTO test_flow VALUES (1, 1, 'Home') (2, 1, 'Gift') (3, 1, 'Exit');
INSERT INTO test_flow VALUES (1, 2, 'Home') (2, 2, 'Home') (3, 2, 'Gift') (4, 2, 'Basket');
INSERT INTO test_flow VALUES (1, 3, 'Gift') (2, 3, 'Home') (3, 3, 'Gift') (4, 3, 'Basket');
```
```SQL
SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'Home', page = 'Home', page = 'Gift') FROM test_flow GROUP BY id;
dt id page
1970-01-01 09:00:01 1 Home // Base point, Matched with Home
1970-01-01 09:00:02 1 Gift // Matched with Gift
1970-01-01 09:00:03 1 Exit // The result
1970-01-01 09:00:01 2 Home // Base point, Matched with Home
1970-01-01 09:00:02 2 Home // Unmatched with Gift
1970-01-01 09:00:03 2 Gift
1970-01-01 09:00:04 2 Basket
1970-01-01 09:00:01 3 Gift // Base point, Unmatched with Home
1970-01-01 09:00:02 3 Home
1970-01-01 09:00:03 3 Gift
1970-01-01 09:00:04 3 Basket
```
**Behavior for `backward` and `tail`**
```SQL
SELECT id, sequenceNextNode('backward', 'tail')(dt, page, page = 'Basket', page = 'Basket', page = 'Gift') FROM test_flow GROUP BY id;
dt id page
1970-01-01 09:00:01 1 Home
1970-01-01 09:00:02 1 Gift
1970-01-01 09:00:03 1 Exit // Base point, Unmatched with Basket
1970-01-01 09:00:01 2 Home
1970-01-01 09:00:02 2 Home // The result
1970-01-01 09:00:03 2 Gift // Matched with Gift
1970-01-01 09:00:04 2 Basket // Base point, Matched with Basket
1970-01-01 09:00:01 3 Gift
1970-01-01 09:00:02 3 Home // The result
1970-01-01 09:00:03 3 Gift // Base point, Matched with Gift
1970-01-01 09:00:04 3 Basket // Base point, Matched with Basket
```
**Behavior for `forward` and `first_match`**
```SQL
SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift', page = 'Gift') FROM test_flow GROUP BY id;
dt id page
1970-01-01 09:00:01 1 Home
1970-01-01 09:00:02 1 Gift // Base point
1970-01-01 09:00:03 1 Exit // The result
1970-01-01 09:00:01 2 Home
1970-01-01 09:00:02 2 Home
1970-01-01 09:00:03 2 Gift // Base point
1970-01-01 09:00:04 2 Basket The result
1970-01-01 09:00:01 3 Gift // Base point
1970-01-01 09:00:02 3 Home // Thre result
1970-01-01 09:00:03 3 Gift
1970-01-01 09:00:04 3 Basket
```
```SQL
SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift', page = 'Gift', page = 'Home') FROM test_flow GROUP BY id;
dt id page
1970-01-01 09:00:01 1 Home
1970-01-01 09:00:02 1 Gift // Base point
1970-01-01 09:00:03 1 Exit // Unmatched with Home
1970-01-01 09:00:01 2 Home
1970-01-01 09:00:02 2 Home
1970-01-01 09:00:03 2 Gift // Base point
1970-01-01 09:00:04 2 Basket // Unmatched with Home
1970-01-01 09:00:01 3 Gift // Base point
1970-01-01 09:00:02 3 Home // Matched with Home
1970-01-01 09:00:03 3 Gift // The result
1970-01-01 09:00:04 3 Basket
```
**Behavior for `backward` and `last_match`**
```SQL
SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', page = 'Gift') FROM test_flow GROUP BY id;
dt id page
1970-01-01 09:00:01 1 Home // The result
1970-01-01 09:00:02 1 Gift // Base point
1970-01-01 09:00:03 1 Exit
1970-01-01 09:00:01 2 Home
1970-01-01 09:00:02 2 Home // The result
1970-01-01 09:00:03 2 Gift // Base point
1970-01-01 09:00:04 2 Basket
1970-01-01 09:00:01 3 Gift
1970-01-01 09:00:02 3 Home // The result
1970-01-01 09:00:03 3 Gift // Base point
1970-01-01 09:00:04 3 Basket
```
```SQL
SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', page = 'Gift', page = 'Home') FROM test_flow GROUP BY id;
dt id page
1970-01-01 09:00:01 1 Home // Matched with Home, the result is null
1970-01-01 09:00:02 1 Gift // Base point
1970-01-01 09:00:03 1 Exit
1970-01-01 09:00:01 2 Home // The result
1970-01-01 09:00:02 2 Home // Matched with Home
1970-01-01 09:00:03 2 Gift // Base point
1970-01-01 09:00:04 2 Basket
1970-01-01 09:00:01 3 Gift // The result
1970-01-01 09:00:02 3 Home // Matched with Home
1970-01-01 09:00:03 3 Gift // Base point
1970-01-01 09:00:04 3 Basket
```
**Behavior for `base_condition`**
```SQL
CREATE TABLE test_flow_basecond
(
`dt` DateTime,
`id` int,
`page` String,
`ref` String
)
ENGINE = MergeTree
PARTITION BY toYYYYMMDD(dt)
ORDER BY id
INSERT INTO test_flow_basecond VALUES (1, 1, 'A', 'ref4') (2, 1, 'A', 'ref3') (3, 1, 'B', 'ref2') (4, 1, 'B', 'ref1');
```
```SQL
SELECT id, sequenceNextNode('forward', 'head')(dt, page, ref = 'ref1', page = 'A') FROM test_flow_basecond GROUP BY id;
dt id page ref
1970-01-01 09:00:01 1 A ref4 // The head can't be base point becasue the ref column of the head unmatched with 'ref1'.
1970-01-01 09:00:02 1 A ref3
1970-01-01 09:00:03 1 B ref2
1970-01-01 09:00:04 1 B ref1
```
```SQL
SELECT id, sequenceNextNode('backward', 'tail')(dt, page, ref = 'ref4', page = 'B') FROM test_flow_basecond GROUP BY id;
dt id page ref
1970-01-01 09:00:01 1 A ref4
1970-01-01 09:00:02 1 A ref3
1970-01-01 09:00:03 1 B ref2
1970-01-01 09:00:04 1 B ref1 // The tail can't be base point becasue the ref column of the tail unmatched with 'ref4'.
```
```SQL
SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, ref = 'ref3', page = 'A') FROM test_flow_basecond GROUP BY id;
dt id page ref
1970-01-01 09:00:01 1 A ref4 // This row can't be base point becasue the ref column unmatched with 'ref3'.
1970-01-01 09:00:02 1 A ref3 // Base point
1970-01-01 09:00:03 1 B ref2 // The result
1970-01-01 09:00:04 1 B ref1
```
```SQL
SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, ref = 'ref2', page = 'B') FROM test_flow_basecond GROUP BY id;
dt id page ref
1970-01-01 09:00:01 1 A ref4
1970-01-01 09:00:02 1 A ref3 // The result
1970-01-01 09:00:03 1 B ref2 // Base point
1970-01-01 09:00:04 1 B ref1 // This row can't be base point becasue the ref column unmatched with 'ref2'.
```

View File

@ -16,6 +16,7 @@ The following operations with [partitions](../../../engines/table-engines/merget
- [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) — Resets the value of a specified column in a partition.
- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) — Resets the specified secondary index in a partition.
- [FREEZE PARTITION](#alter_freeze-partition) — Creates a backup of a partition.
- [UNFREEZE PARTITION](#alter_unfreeze-partition) — Removes a backup of a partition.
- [FETCH PARTITION\|PART](#alter_fetch-partition) — Downloads a part or partition from another server.
- [MOVE PARTITION\|PART](#alter_move-partition) — Move partition/data part to another disk or volume.
@ -160,7 +161,7 @@ This query creates a local backup of a specified partition. If the `PARTITION` c
!!! note "Note"
The entire backup process is performed without stopping the server.
Note that for old-styled tables you can specify the prefix of the partition name (for example, 2019) - then the query creates the backup for all the corresponding partitions. Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
Note that for old-styled tables you can specify the prefix of the partition name (for example, `2019`) - then the query creates the backup for all the corresponding partitions. Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
At the time of execution, for a data snapshot, the query creates hardlinks to a table data. Hardlinks are placed in the directory `/var/lib/clickhouse/shadow/N/...`, where:
@ -170,7 +171,7 @@ At the time of execution, for a data snapshot, the query creates hardlinks to a
!!! note "Note"
If you use [a set of disks for data storage in a table](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes), the `shadow/N` directory appears on every disk, storing data parts that matched by the `PARTITION` expression.
The same structure of directories is created inside the backup as inside `/var/lib/clickhouse/`. The query performs chmod for all files, forbidding writing into them.
The same structure of directories is created inside the backup as inside `/var/lib/clickhouse/`. The query performs `chmod` for all files, forbidding writing into them.
After creating the backup, you can copy the data from `/var/lib/clickhouse/shadow/` to the remote server and then delete it from the local server. Note that the `ALTER t FREEZE PARTITION` query is not replicated. It creates a local backup only on the local server.
@ -188,6 +189,14 @@ Restoring from a backup does not require stopping the server.
For more information about backups and restoring data, see the [Data Backup](../../../operations/backup.md) section.
## UNFREEZE PARTITION {#alter_unfreeze-partition}
``` sql
ALTER TABLE 'table_name' UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name'
```
Removes `freezed` partitions with the specified name from the disk. If the `PARTITION` clause is omitted, the query removes the backup of all partitions at once.
## CLEAR INDEX IN PARTITION {#alter_clear-index-partition}
``` sql

View File

@ -120,6 +120,19 @@ Eсли суммарное число активных кусков во все
Команда `Insert` создает один или несколько блоков (кусков). При вставке в Replicated таблицы ClickHouse для [дедупликации вставок](../../engines/table-engines/mergetree-family/replication.md) записывает в Zookeeper хеш-суммы созданных кусков. Но хранятся хеш-суммы не всех кусков, а только последние `replicated_deduplication_window`. Наиболее старые хеш-суммы удаляются из Zookeeper.
Большое число `replicated_deduplication_window` замедляет `Insert`-ы. Хеш-сумма рассчитывается от композиции имен и типов полей, а также данных вставленного куска (потока байт).
## non_replicated_deduplication_window {#non-replicated-deduplication-window}
Количество последних вставленных блоков в нереплицированной [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) таблице, для которых хранятся хеш-суммы для проверки дубликатов.
Возможные значения:
- Положительное целое число.
- 0 (дедупликация отключена).
Значение по умолчанию: 0.
Используется механизм дедупликации, аналогичный реплицированным таблицам (см. описание настройки [replicated_deduplication_window](#replicated-deduplication-window)). Хеш-суммы вставленных кусков записываются в локальный файл на диске.
## replicated_deduplication_window_seconds {#replicated-deduplication-window-seconds}
Число секунд, после которых хеш-суммы вставленных блоков удаляются из Zookeeper.

View File

@ -16,6 +16,7 @@ toc_title: PARTITION
- [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) — удалить все значения в столбце для заданной партиции;
- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) — очистить построенные вторичные индексы для заданной партиции;
- [FREEZE PARTITION](#alter_freeze-partition) — создать резервную копию партиции;
- [UNFREEZE PARTITION](#alter_unfreeze-partition) — удалить резервную копию партиции;
- [FETCH PARTITION](#alter_fetch-partition) — скачать партицию с другого сервера;
- [MOVE PARTITION\|PART](#alter_move-partition) — переместить партицию/кускок на другой диск или том.
@ -170,9 +171,9 @@ ALTER TABLE table_name FREEZE [PARTITION partition_expr]
!!! note "Примечание"
Создание резервной копии не требует остановки сервера.
Для таблиц старого стиля имя партиций можно задавать в виде префикса (например, 2019). В этом случае резервные копии будут созданы для всех соответствующих партиций. Подробнее о том, как корректно задать имя партиции, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr).
Для таблиц старого стиля имя партиций можно задавать в виде префикса (например, `2019`). В этом случае, резервные копии будут созданы для всех соответствующих партиций. Подробнее о том, как корректно задать имя партиции, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr).
Запрос делает следующее — для текущего состояния таблицы он формирует жесткие ссылки на данные в этой таблице. Ссылки размещаются в директории `/var/lib/clickhouse/shadow/N/...`, где:
Запрос формирует для текущего состояния таблицы жесткие ссылки на данные в этой таблице. Ссылки размещаются в директории `/var/lib/clickhouse/shadow/N/...`, где:
- `/var/lib/clickhouse/` — рабочая директория ClickHouse, заданная в конфигурационном файле;
- `N` — инкрементальный номер резервной копии.
@ -180,11 +181,11 @@ ALTER TABLE table_name FREEZE [PARTITION partition_expr]
!!! note "Примечание"
При использовании [нескольких дисков для хранения данных таблицы](../../statements/alter/index.md#table_engine-mergetree-multiple-volumes) директория `shadow/N` появляется на каждом из дисков, на которых были куски, попавшие под выражение `PARTITION`.
Структура директорий внутри резервной копии такая же, как внутри `/var/lib/clickhouse/`. Запрос выполнит chmod для всех файлов, запрещая запись в них.
Структура директорий внутри резервной копии такая же, как внутри `/var/lib/clickhouse/`. Запрос выполнит `chmod` для всех файлов, запрещая запись в них.
Обратите внимание, запрос `ALTER TABLE t FREEZE PARTITION` не реплицируется. Он создает резервную копию только на локальном сервере. После создания резервной копии данные из `/var/lib/clickhouse/shadow/` можно скопировать на удалённый сервер, а локальную копию удалить.
Резервная копия создается почти мгновенно (однако сначала запрос дожидается завершения всех запросов, которые выполняются для соответствующей таблицы).
Резервная копия создается почти мгновенно (однако, сначала запрос дожидается завершения всех запросов, которые выполняются для соответствующей таблицы).
`ALTER TABLE t FREEZE PARTITION` копирует только данные, но не метаданные таблицы. Чтобы сделать резервную копию метаданных таблицы, скопируйте файл `/var/lib/clickhouse/metadata/database/table.sql`
@ -198,6 +199,14 @@ ALTER TABLE table_name FREEZE [PARTITION partition_expr]
Подробнее о резервном копировании и восстановлении данных читайте в разделе [Резервное копирование данных](../../../operations/backup.md).
## UNFREEZE PARTITION {#alter_unfreeze-partition}
``` sql
ALTER TABLE 'table_name' UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name'
```
Удаляет с диска "замороженные" партиции с указанным именем. Если секция `PARTITION` опущена, запрос удаляет резервную копию всех партиций сразу.
## FETCH PARTITION {#alter_fetch-partition}
``` sql

View File

@ -94,7 +94,7 @@ INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def')
Обозначает, что значение отсутствует.
Чтобы в поле таблицы можно было хранить `NULL`, оно должно быть типа [Nullable](../sql-reference/syntax.md).
Чтобы в поле таблицы можно было хранить `NULL`, оно должно быть типа [Nullable](../sql-reference/data-types/nullable.md).
В зависимости от формата данных (входных или выходных) `NULL` может иметь различное представление. Подробнее смотрите в документации для [форматов данных](../interfaces/formats.md#formats).

View File

@ -6,6 +6,7 @@
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
namespace DB
{
namespace ErrorCodes

View File

@ -29,11 +29,11 @@ AggregateFunctionPtr createAggregateFunctionDeltaSumTimestamp(
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!isInteger(arguments[0]) && !isFloat(arguments[0]) && !isDateOrDateTime(arguments[0]))
if (!isInteger(arguments[0]) && !isFloat(arguments[0]) && !isDate(arguments[0]) && !isDateTime(arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument for aggregate function " +
name + ", must be Int, Float, Date, DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!isInteger(arguments[1]) && !isFloat(arguments[1]) && !isDateOrDateTime(arguments[1]))
if (!isInteger(arguments[1]) && !isFloat(arguments[1]) && !isDate(arguments[1]) && !isDateTime(arguments[1]))
throw Exception("Illegal type " + arguments[1]->getName() + " of argument for aggregate function " +
name + ", must be Int, Float, Date, DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

View File

@ -53,7 +53,7 @@ public:
{
WhichDataType which{arguments.back()};
if (which.isNativeUInt() || which.isDateOrDateTime())
if (which.isNativeUInt() || which.isDate() || which.isDateTime() || which.isDateTime64())
{
UInt64 begin = params[params.size() - 3].safeGet<UInt64>();
UInt64 end = params[params.size() - 2].safeGet<UInt64>();

View File

@ -38,7 +38,7 @@ namespace
for (const auto & arg : args)
{
if (!isNativeNumber(arg) && !isDateOrDateTime(arg))
if (!isNativeNumber(arg) && !isDate(arg) && !isDateTime(arg) && !isDateTime64(arg))
throw Exception(
"Illegal type " + arg->getName() + " of argument of aggregate function " + name
+ ", must be Number, Date, DateTime or DateTime64",

View File

@ -0,0 +1,144 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionSequenceNextNode.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <Core/Settings.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeNullable.h>
#include <Interpreters/Context.h>
#include <Common/CurrentThread.h>
#include <ext/range.h>
namespace DB
{
constexpr size_t max_events_size = 64;
constexpr size_t min_required_args = 3;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
extern const int UNKNOWN_AGGREGATE_FUNCTION;
}
namespace
{
template <typename T>
inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl(
const DataTypePtr data_type, const DataTypes & argument_types, SequenceDirection direction, SequenceBase base)
{
return std::make_shared<SequenceNextNodeImpl<T, NodeString<max_events_size>>>(
data_type, argument_types, base, direction, min_required_args);
}
AggregateFunctionPtr
createAggregateFunctionSequenceNode(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
if (settings == nullptr || !settings->allow_experimental_funnel_functions)
{
throw Exception(
"Aggregate function " + name + " is experimental. Set `allow_experimental_funnel_functions` setting to enable it",
ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION);
}
if (parameters.size() < 2)
throw Exception("Aggregate function '" + name + "' requires 2 parameters (direction, head)",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
auto expected_param_type = Field::Types::Which::String;
if (parameters.at(0).getType() != expected_param_type || parameters.at(1).getType() != expected_param_type)
throw Exception("Aggregate function '" + name + "' requires 'String' parameters",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
String param_dir = parameters.at(0).safeGet<String>();
std::unordered_map<std::string, SequenceDirection> seq_dir_mapping{
{"forward", SequenceDirection::Forward},
{"backward", SequenceDirection::Backward},
};
if (!seq_dir_mapping.contains(param_dir))
throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_dir, ErrorCodes::BAD_ARGUMENTS};
SequenceDirection direction = seq_dir_mapping[param_dir];
String param_base = parameters.at(1).safeGet<String>();
std::unordered_map<std::string, SequenceBase> seq_base_mapping{
{"head", SequenceBase::Head},
{"tail", SequenceBase::Tail},
{"first_match", SequenceBase::FirstMatch},
{"last_match", SequenceBase::LastMatch},
};
if (!seq_base_mapping.contains(param_base))
throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_base, ErrorCodes::BAD_ARGUMENTS};
SequenceBase base = seq_base_mapping[param_base];
if ((base == SequenceBase::Head && direction == SequenceDirection::Backward) ||
(base == SequenceBase::Tail && direction == SequenceDirection::Forward))
throw Exception(fmt::format(
"Invalid argument combination of '{}' with '{}'", param_base, param_dir), ErrorCodes::BAD_ARGUMENTS);
if (argument_types.size() < min_required_args)
throw Exception("Aggregate function " + name + " requires at least " + toString(min_required_args) + " arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
bool is_base_match_type = base == SequenceBase::FirstMatch || base == SequenceBase::LastMatch;
if (is_base_match_type && argument_types.size() < min_required_args + 1)
throw Exception(
"Aggregate function " + name + " requires at least " + toString(min_required_args + 1) + " arguments when base is first_match or last_match.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (argument_types.size() > max_events_size + min_required_args)
throw Exception(fmt::format(
"Aggregate function '{}' requires at most {} (timestamp, value_column, ...{} events) arguments.",
name, max_events_size + min_required_args, max_events_size), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (const auto * cond_arg = argument_types[2].get(); cond_arg && !isUInt8(cond_arg))
throw Exception("Illegal type " + cond_arg->getName() + " of third argument of aggregate function "
+ name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
for (const auto i : ext::range(min_required_args, argument_types.size()))
{
const auto * cond_arg = argument_types[i].get();
if (!isUInt8(cond_arg))
throw Exception(fmt::format(
"Illegal type '{}' of {} argument of aggregate function '{}', must be UInt8", cond_arg->getName(), i + 1, name),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
if (WhichDataType(argument_types[1].get()).idx != TypeIndex::String)
throw Exception{"Illegal type " + argument_types[1].get()->getName()
+ " of second argument of aggregate function " + name + ", must be String",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
DataTypePtr data_type = makeNullable(argument_types[1]);
WhichDataType timestamp_type(argument_types[0].get());
if (timestamp_type.idx == TypeIndex::UInt8)
return createAggregateFunctionSequenceNodeImpl<UInt8>(data_type, argument_types, direction, base);
if (timestamp_type.idx == TypeIndex::UInt16)
return createAggregateFunctionSequenceNodeImpl<UInt16>(data_type, argument_types, direction, base);
if (timestamp_type.idx == TypeIndex::UInt32)
return createAggregateFunctionSequenceNodeImpl<UInt32>(data_type, argument_types, direction, base);
if (timestamp_type.idx == TypeIndex::UInt64)
return createAggregateFunctionSequenceNodeImpl<UInt64>(data_type, argument_types, direction, base);
if (timestamp_type.isDate())
return createAggregateFunctionSequenceNodeImpl<DataTypeDate::FieldType>(data_type, argument_types, direction, base);
if (timestamp_type.isDateTime())
return createAggregateFunctionSequenceNodeImpl<DataTypeDateTime::FieldType>(data_type, argument_types, direction, base);
throw Exception{"Illegal type " + argument_types.front().get()->getName()
+ " of first argument of aggregate function " + name + ", must be Unsigned Number, Date, DateTime",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
}
void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory)
{
AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false };
factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNode, properties });
}
}

View File

@ -0,0 +1,426 @@
#pragma once
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnNullable.h>
#include <Common/ArenaAllocator.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/AggregateFunctionNull.h>
#include <type_traits>
#include <bitset>
namespace DB
{
struct Settings;
enum class SequenceDirection
{
Forward,
Backward,
};
enum SequenceBase
{
Head,
Tail,
FirstMatch,
LastMatch,
};
/// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl
template <typename Node, size_t MaxEventsSize>
struct NodeBase
{
UInt64 size; /// size of payload
DataTypeDateTime::FieldType event_time;
std::bitset<MaxEventsSize> events_bitset;
bool can_be_base;
char * data() { return reinterpret_cast<char *>(this) + sizeof(Node); }
const char * data() const { return reinterpret_cast<const char *>(this) + sizeof(Node); }
Node * clone(Arena * arena) const
{
return reinterpret_cast<Node *>(
const_cast<char *>(arena->alignedInsert(reinterpret_cast<const char *>(this), sizeof(Node) + size, alignof(Node))));
}
void write(WriteBuffer & buf) const
{
writeVarUInt(size, buf);
buf.write(data(), size);
writeBinary(event_time, buf);
UInt64 ulong_bitset = events_bitset.to_ulong();
writeBinary(ulong_bitset, buf);
writeBinary(can_be_base, buf);
}
static Node * read(ReadBuffer & buf, Arena * arena)
{
UInt64 size;
readVarUInt(size, buf);
Node * node = reinterpret_cast<Node *>(arena->alignedAlloc(sizeof(Node) + size, alignof(Node)));
node->size = size;
buf.read(node->data(), size);
readBinary(node->event_time, buf);
UInt64 ulong_bitset;
readBinary(ulong_bitset, buf);
node->events_bitset = ulong_bitset;
readBinary(node->can_be_base, buf);
return node;
}
};
/// It stores String, timestamp, bitset of matched events.
template <size_t MaxEventsSize>
struct NodeString : public NodeBase<NodeString<MaxEventsSize>, MaxEventsSize>
{
using Node = NodeString<MaxEventsSize>;
static Node * allocate(const IColumn & column, size_t row_num, Arena * arena)
{
StringRef string = assert_cast<const ColumnString &>(column).getDataAt(row_num);
Node * node = reinterpret_cast<Node *>(arena->alignedAlloc(sizeof(Node) + string.size, alignof(Node)));
node->size = string.size;
memcpy(node->data(), string.data, string.size);
return node;
}
void insertInto(IColumn & column)
{
assert_cast<ColumnString &>(column).insertData(this->data(), this->size);
}
bool compare(const Node * rhs) const
{
auto cmp = strncmp(this->data(), rhs->data(), std::min(this->size, rhs->size));
return (cmp == 0) ? this->size < rhs->size : cmp < 0;
}
};
/// TODO : Support other types than string
template <typename Node>
struct SequenceNextNodeGeneralData
{
using Allocator = MixedAlignedArenaAllocator<alignof(Node *), 4096>;
using Array = PODArray<Node *, 32, Allocator>;
Array value;
bool sorted = false;
struct Comparator final
{
bool operator()(const Node * lhs, const Node * rhs) const
{
return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time < rhs->event_time;
}
};
void sort()
{
if (!sorted)
{
std::stable_sort(std::begin(value), std::end(value), Comparator{});
sorted = true;
}
}
};
/// Implementation of sequenceFirstNode
template <typename T, typename Node>
class SequenceNextNodeImpl final
: public IAggregateFunctionDataHelper<SequenceNextNodeGeneralData<Node>, SequenceNextNodeImpl<T, Node>>
{
using Self = SequenceNextNodeImpl<T, Node>;
using Data = SequenceNextNodeGeneralData<Node>;
static Data & data(AggregateDataPtr place) { return *reinterpret_cast<Data *>(place); }
static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast<const Data *>(place); }
static constexpr size_t base_cond_column_idx = 2;
static constexpr size_t event_column_idx = 1;
SequenceBase seq_base_kind;
SequenceDirection seq_direction;
const size_t min_required_args;
DataTypePtr & data_type;
UInt8 events_size;
UInt64 max_elems;
public:
SequenceNextNodeImpl(
const DataTypePtr & data_type_,
const DataTypes & arguments,
SequenceBase seq_base_kind_,
SequenceDirection seq_direction_,
size_t min_required_args_,
UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
: IAggregateFunctionDataHelper<SequenceNextNodeGeneralData<Node>, Self>({data_type_}, {})
, seq_base_kind(seq_base_kind_)
, seq_direction(seq_direction_)
, min_required_args(min_required_args_)
, data_type(this->argument_types[0])
, events_size(arguments.size() - min_required_args)
, max_elems(max_elems_)
{
}
String getName() const override { return "sequenceNextNode"; }
DataTypePtr getReturnType() const override { return data_type; }
AggregateFunctionPtr getOwnNullAdapter(
const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params,
const AggregateFunctionProperties &) const override
{
/// Even though some values are mapped to aggregating key, it could return nulls for the below case.
/// aggregated events: [A -> B -> C]
/// events to find: [C -> D]
/// [C -> D] is not matched to 'A -> B -> C' so that it returns null.
return std::make_shared<AggregateFunctionNullVariadic<false, false, true>>(nested_function, arguments, params);
}
void insert(Data & a, const Node * v, Arena * arena) const
{
++a.total_values;
a.value.push_back(v->clone(arena), arena);
}
void create(AggregateDataPtr place) const override
{
new (place) Data;
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
Node * node = Node::allocate(*columns[event_column_idx], row_num, arena);
const auto timestamp = assert_cast<const ColumnVector<T> *>(columns[0])->getData()[row_num];
/// The events_bitset variable stores matched events in the form of bitset.
/// Each Nth-bit indicates that the Nth-event are matched.
/// For example, event1 and event3 is matched then the values of events_bitset is 0x00000005.
/// 0x00000000
/// + 1 (bit of event1)
/// + 4 (bit of event3)
node->events_bitset.reset();
for (UInt8 i = 0; i < events_size; ++i)
if (assert_cast<const ColumnVector<UInt8> *>(columns[min_required_args + i])->getData()[row_num])
node->events_bitset.set(i);
node->event_time = timestamp;
node->can_be_base = assert_cast<const ColumnVector<UInt8> *>(columns[base_cond_column_idx])->getData()[row_num];
data(place).value.push_back(node, arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
if (data(rhs).value.empty())
return;
if (data(place).value.size() >= max_elems)
return;
auto & a = data(place).value;
auto & b = data(rhs).value;
const auto a_size = a.size();
const UInt64 new_elems = std::min(data(rhs).value.size(), static_cast<size_t>(max_elems) - data(place).value.size());
for (UInt64 i = 0; i < new_elems; ++i)
a.push_back(b[i]->clone(arena), arena);
/// Either sort whole container or do so partially merging ranges afterwards
using Comparator = typename SequenceNextNodeGeneralData<Node>::Comparator;
if (!data(place).sorted && !data(rhs).sorted)
std::stable_sort(std::begin(a), std::end(a), Comparator{});
else
{
const auto begin = std::begin(a);
const auto middle = std::next(begin, a_size);
const auto end = std::end(a);
if (!data(place).sorted)
std::stable_sort(begin, middle, Comparator{});
if (!data(rhs).sorted)
std::stable_sort(middle, end, Comparator{});
std::inplace_merge(begin, middle, end, Comparator{});
}
data(place).sorted = true;
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override
{
/// Temporarily do a const_cast to sort the values. It helps to reduce the computational burden on the initiator node.
this->data(const_cast<AggregateDataPtr>(place)).sort();
writeBinary(data(place).sorted, buf);
auto & value = data(place).value;
size_t size = std::min(static_cast<size_t>(events_size + 1), value.size());
switch (seq_base_kind)
{
case SequenceBase::Head:
writeVarUInt(size, buf);
for (size_t i = 0; i < size; ++i)
value[i]->write(buf);
break;
case SequenceBase::Tail:
writeVarUInt(size, buf);
for (size_t i = 0; i < size; ++i)
value[value.size() - size + i]->write(buf);
break;
case SequenceBase::FirstMatch:
case SequenceBase::LastMatch:
writeVarUInt(value.size(), buf);
for (auto & node : value)
node->write(buf);
break;
}
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override
{
readBinary(data(place).sorted, buf);
UInt64 size;
readVarUInt(size, buf);
if (unlikely(size == 0))
return;
auto & value = data(place).value;
value.resize(size, arena);
for (UInt64 i = 0; i < size; ++i)
value[i] = Node::read(buf, arena);
}
inline std::optional<size_t> getBaseIndex(Data & data) const
{
if (data.value.size() == 0)
return {};
switch (seq_base_kind)
{
case SequenceBase::Head:
if (data.value[0]->can_be_base)
return 0;
break;
case SequenceBase::Tail:
if (data.value[data.value.size() - 1]->can_be_base)
return data.value.size() - 1;
break;
case SequenceBase::FirstMatch:
for (size_t i = 0; i < data.value.size(); ++i)
{
if (data.value[i]->events_bitset.test(0) && data.value[i]->can_be_base)
return i;
}
break;
case SequenceBase::LastMatch:
for (size_t i = 0; i < data.value.size(); ++i)
{
auto reversed_i = data.value.size() - i - 1;
if (data.value[reversed_i]->events_bitset.test(0) && data.value[reversed_i]->can_be_base)
return reversed_i;
}
break;
}
return {};
}
/// This method returns an index of next node that matched the events.
/// matched events in the chain of events are represented as a bitmask.
/// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on.
UInt32 getNextNodeIndex(Data & data) const
{
const UInt32 unmatched_idx = data.value.size();
if (data.value.size() <= events_size)
return unmatched_idx;
data.sort();
std::optional<size_t> base_opt = getBaseIndex(data);
if (!base_opt.has_value())
return unmatched_idx;
UInt32 base = static_cast<UInt32>(base_opt.value());
if (events_size == 0)
return data.value.size() > 0 ? base : unmatched_idx;
UInt32 i = 0;
switch (seq_direction)
{
case SequenceDirection::Forward:
for (i = 0; i < events_size && base + i < data.value.size(); ++i)
if (!data.value[base + i]->events_bitset.test(i))
break;
return (i == events_size) ? base + i : unmatched_idx;
case SequenceDirection::Backward:
for (i = 0; i < events_size && i < base; ++i)
if (!data.value[base - i]->events_bitset.test(i))
break;
return (i == events_size) ? base - i : unmatched_idx;
}
__builtin_unreachable();
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & value = data(place).value;
UInt32 event_idx = getNextNodeIndex(this->data(place));
if (event_idx < value.size())
{
ColumnNullable & to_concrete = assert_cast<ColumnNullable &>(to);
value[event_idx]->insertInto(to_concrete.getNestedColumn());
to_concrete.getNullMapData().push_back(0);
}
else
{
to.insertDefault();
}
}
bool allocatesMemoryInArena() const override { return true; }
};
}

View File

@ -17,7 +17,6 @@ namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int UNKNOWN_AGGREGATE_FUNCTION;
}
namespace
@ -25,15 +24,8 @@ namespace
template <template <typename> class Data>
AggregateFunctionPtr
createAggregateFunctionWindowFunnel(const std::string & name, const DataTypes & arguments, const Array & params, const Settings * settings)
createAggregateFunctionWindowFunnel(const std::string & name, const DataTypes & arguments, const Array & params, const Settings *)
{
if (settings == nullptr || !settings->allow_experimental_funnel_functions)
{
throw Exception(
"Aggregate function " + name + " is experimental. Set `allow_experimental_funnel_functions` setting to enable it",
ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION);
}
if (params.empty())
throw Exception{"Aggregate function " + name + " requires at least one parameter: <window>, [option, [option, ...]]", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};

View File

@ -48,6 +48,7 @@ void registerAggregateFunctionRankCorrelation(AggregateFunctionFactory &);
void registerAggregateFunctionMannWhitney(AggregateFunctionFactory &);
void registerAggregateFunctionWelchTTest(AggregateFunctionFactory &);
void registerAggregateFunctionStudentTTest(AggregateFunctionFactory &);
void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory &);
class AggregateFunctionCombinatorFactory;
void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &);
@ -109,6 +110,7 @@ void registerAggregateFunctions()
registerAggregateFunctionAggThrow(factory);
registerAggregateFunctionRankCorrelation(factory);
registerAggregateFunctionMannWhitney(factory);
registerAggregateFunctionSequenceNextNode(factory);
registerAggregateFunctionWelchTTest(factory);
registerAggregateFunctionStudentTTest(factory);

View File

@ -45,6 +45,7 @@ SRCS(
AggregateFunctionRetention.cpp
AggregateFunctionSegmentLengthSum.cpp
AggregateFunctionSequenceMatch.cpp
AggregateFunctionSequenceNextNode.cpp
AggregateFunctionSimpleLinearRegression.cpp
AggregateFunctionSimpleState.cpp
AggregateFunctionState.cpp

View File

@ -35,7 +35,6 @@ ColumnNullable::ColumnNullable(MutableColumnPtr && nested_column_, MutableColumn
throw Exception{"ColumnNullable cannot have constant null map", ErrorCodes::ILLEGAL_COLUMN};
}
void ColumnNullable::updateHashWithValue(size_t n, SipHash & hash) const
{
const auto & arr = getNullMapData();

108
src/Common/Throttler.cpp Normal file
View File

@ -0,0 +1,108 @@
#include <Common/Throttler.h>
#include <Common/ProfileEvents.h>
#include <Common/Exception.h>
#include <Common/Stopwatch.h>
#include <IO/WriteHelpers.h>
#include <cmath>
namespace ProfileEvents
{
extern const Event ThrottlerSleepMicroseconds;
}
namespace DB
{
namespace ErrorCodes
{
extern const int LIMIT_EXCEEDED;
}
/// Just 10^9.
static constexpr auto NS = 1000000000UL;
/// Tracking window. Actually the size is not really important. We just want to avoid
/// throttles when there are no actions for a long period time.
static const double window_ns = 7UL * NS;
void Throttler::add(size_t amount)
{
size_t new_count;
/// This outer variable is always equal to smoothed_speed.
/// We use to avoid race condition.
double current_speed = 0;
{
std::lock_guard lock(mutex);
auto now = clock_gettime_ns();
/// If prev_ns is equal to zero (first `add` call) we known nothing about speed
/// and don't track anything.
if (max_speed && prev_ns != 0)
{
/// Time spent to process the amount of bytes
double time_spent = now - prev_ns;
/// The speed in bytes per second is equal to amount / time_spent in seconds
auto new_speed = amount / (time_spent / NS);
/// We want to make old values of speed less important for our smoothed value
/// so we decay it's value with coef.
auto decay_coeff = std::pow(0.5, time_spent / window_ns);
/// Weighted average between previous and new speed
smoothed_speed = smoothed_speed * decay_coeff + (1 - decay_coeff) * new_speed;
current_speed = smoothed_speed;
}
count += amount;
new_count = count;
prev_ns = now;
}
if (limit && new_count > limit)
throw Exception(limit_exceeded_exception_message + std::string(" Maximum: ") + toString(limit), ErrorCodes::LIMIT_EXCEEDED);
if (max_speed && current_speed > max_speed)
{
/// If we was too fast then we have to sleep until our smoothed speed became <= max_speed
int64_t sleep_time = -window_ns * std::log2(max_speed / current_speed);
if (sleep_time > 0)
{
accumulated_sleep += sleep_time;
sleepForNanoseconds(sleep_time);
accumulated_sleep -= sleep_time;
ProfileEvents::increment(ProfileEvents::ThrottlerSleepMicroseconds, sleep_time / 1000UL);
}
}
if (parent)
parent->add(amount);
}
void Throttler::reset()
{
std::lock_guard lock(mutex);
count = 0;
accumulated_sleep = 0;
smoothed_speed = 0;
prev_ns = 0;
}
bool Throttler::isThrottling() const
{
if (accumulated_sleep != 0)
return true;
if (parent)
return parent->isThrottling();
return false;
}
}

View File

@ -2,32 +2,16 @@
#include <mutex>
#include <memory>
#include <Common/Stopwatch.h>
#include <Common/Exception.h>
#include <Common/ProfileEvents.h>
#include <common/sleep.h>
#include <IO/WriteHelpers.h>
namespace ProfileEvents
{
extern const Event ThrottlerSleepMicroseconds;
}
#include <atomic>
namespace DB
{
namespace ErrorCodes
{
extern const int LIMIT_EXCEEDED;
}
/** Allows you to limit the speed of something (in entities per second) using sleep.
* Specifics of work:
* - only the average speed is considered, from the moment of the first call of `add` function;
* if there were periods with low speed, then during some time after them, the speed will be higher;
* Tracks exponentially (pow of 1/2) smoothed speed with hardcoded window.
* See more comments in .cpp file.
*
* Also allows you to set a limit on the maximum number of entities. If exceeded, an exception will be thrown.
*/
@ -41,49 +25,9 @@ public:
const std::shared_ptr<Throttler> & parent_ = nullptr)
: max_speed(max_speed_), limit(limit_), limit_exceeded_exception_message(limit_exceeded_exception_message_), parent(parent_) {}
void add(const size_t amount)
{
size_t new_count;
UInt64 elapsed_ns = 0;
{
std::lock_guard lock(mutex);
if (max_speed)
{
if (0 == count)
{
watch.start();
elapsed_ns = 0;
}
else
elapsed_ns = watch.elapsed();
}
count += amount;
new_count = count;
}
if (limit && new_count > limit)
throw Exception(limit_exceeded_exception_message + std::string(" Maximum: ") + toString(limit), ErrorCodes::LIMIT_EXCEEDED);
if (max_speed)
{
/// How much time to wait for the average speed to become `max_speed`.
UInt64 desired_ns = new_count * 1000000000 / max_speed;
if (desired_ns > elapsed_ns)
{
UInt64 sleep_ns = desired_ns - elapsed_ns;
sleepForNanoseconds(sleep_ns);
ProfileEvents::increment(ProfileEvents::ThrottlerSleepMicroseconds, sleep_ns / 1000UL);
}
}
if (parent)
parent->add(amount);
}
/// Calculates the smoothed speed, sleeps if required and throws exception on
/// limit overflow.
void add(size_t amount);
/// Not thread safe
void setParent(const std::shared_ptr<Throttler> & parent_)
@ -91,21 +35,23 @@ public:
parent = parent_;
}
void reset()
{
std::lock_guard lock(mutex);
/// Reset all throttlers internal stats
void reset();
count = 0;
watch.reset();
}
/// Is throttler already accumulated some sleep time and throttling.
bool isThrottling() const;
private:
size_t count = 0;
const size_t max_speed = 0;
const UInt64 limit = 0; /// 0 - not limited.
size_t count{0};
const size_t max_speed{0};
const uint64_t limit{0}; /// 0 - not limited.
const char * limit_exceeded_exception_message = nullptr;
Stopwatch watch {CLOCK_MONOTONIC_COARSE};
std::mutex mutex;
std::atomic<uint64_t> accumulated_sleep{0};
/// Smoothed value of current speed. Updated in `add` method.
double smoothed_speed{0};
/// previous `add` call time (in nanoseconds)
uint64_t prev_ns{0};
/// Used to implement a hierarchy of throttlers
std::shared_ptr<Throttler> parent;

View File

@ -80,6 +80,7 @@ SRCS(
ThreadPool.cpp
ThreadProfileEvents.cpp
ThreadStatus.cpp
Throttler.cpp
TimerDescriptor.cpp
TraceCollector.cpp
UTF8Helpers.cpp

View File

@ -83,6 +83,8 @@ class IColumn;
M(UInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables, dns cache updates. Only has meaning at server startup.", 0) \
M(UInt64, background_message_broker_schedule_pool_size, 16, "Number of threads performing background tasks for message streaming. Only has meaning at server startup.", 0) \
M(UInt64, background_distributed_schedule_pool_size, 16, "Number of threads performing background tasks for distributed sends. Only has meaning at server startup.", 0) \
M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited. Only has meaning at server startup.", 0) \
M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited. Only has meaning at server startup.", 0) \
\
M(Milliseconds, distributed_directory_monitor_sleep_time_ms, 100, "Sleep time for StorageDistributed DirectoryMonitors, in case of any errors delay grows exponentially.", 0) \
M(Milliseconds, distributed_directory_monitor_max_sleep_time_ms, 30000, "Maximum sleep time for StorageDistributed DirectoryMonitors, it limits exponential growth too.", 0) \
@ -468,13 +470,15 @@ class IColumn;
M(UInt64, limit, 0, "Limit on read rows from the most 'end' result for select query, default 0 means no limit length", 0) \
M(UInt64, offset, 0, "Offset on read rows from the most 'end' result for select query", 0) \
\
/** Experimental functions */ \
M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \
\
\
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
M(UInt64, max_memory_usage_for_all_queries, 0, "Obsolete setting, does nothing.", 0) \
M(UInt64, multiple_joins_rewriter_version, 0, "Obsolete setting, does nothing.", 0) \
M(Bool, enable_debug_queries, false, "Obsolete setting, does nothing.", 0) \
M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing.", 0) \
M(Bool, allow_experimental_funnel_functions, true, "Obsolete setting, does nothing.", 0) \
M(Bool, allow_experimental_bigint_types, true, "Obsolete setting, does nothing.", 0) \
M(HandleKafkaErrorMode, handle_kafka_error_mode, HandleKafkaErrorMode::DEFAULT, "Obsolete setting, does nothing.", 0) \
M(Bool, database_replicated_ddl_output, true, "Obsolete setting, does nothing.", 0) \

View File

@ -324,7 +324,6 @@ struct WhichDataType
constexpr bool isDate() const { return idx == TypeIndex::Date; }
constexpr bool isDateTime() const { return idx == TypeIndex::DateTime; }
constexpr bool isDateTime64() const { return idx == TypeIndex::DateTime64; }
constexpr bool isDateOrDateTime() const { return isDate() || isDateTime() || isDateTime64(); }
constexpr bool isString() const { return idx == TypeIndex::String; }
constexpr bool isFixedString() const { return idx == TypeIndex::FixedString; }
@ -348,8 +347,6 @@ struct WhichDataType
template <typename T>
inline bool isDate(const T & data_type) { return WhichDataType(data_type).isDate(); }
template <typename T>
inline bool isDateOrDateTime(const T & data_type) { return WhichDataType(data_type).isDateOrDateTime(); }
template <typename T>
inline bool isDateTime(const T & data_type) { return WhichDataType(data_type).isDateTime(); }
template <typename T>
inline bool isDateTime64(const T & data_type) { return WhichDataType(data_type).isDateTime64(); }
@ -413,7 +410,7 @@ template <typename T>
inline bool isColumnedAsNumber(const T & data_type)
{
WhichDataType which(data_type);
return which.isInt() || which.isUInt() || which.isFloat() || which.isDateOrDateTime() || which.isUUID();
return which.isInt() || which.isUInt() || which.isFloat() || which.isDate() || which.isDateTime() || which.isDateTime64() || which.isUUID();
}
template <typename T>

View File

@ -171,7 +171,7 @@ static inline llvm::Constant * getColumnNativeValue(llvm::IRBuilderBase & builde
{
return llvm::ConstantFP::get(type, assert_cast<const ColumnVector<Float64> &>(column).getElement(index));
}
else if (column_data_type.isNativeUInt() || column_data_type.isDateOrDateTime())
else if (column_data_type.isNativeUInt() || column_data_type.isDate() || column_data_type.isDateTime() || column_data_type.isDateTime64())
{
return llvm::ConstantInt::get(type, column.getUInt(index));
}

View File

@ -241,7 +241,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
for (const auto & type : types)
{
if (isDateOrDateTime(type))
if (isDate(type) || isDateTime(type) || isDateTime64(type))
have_date_or_datetime = true;
else
all_date_or_datetime = false;

View File

@ -599,8 +599,8 @@ class FunctionBinaryArithmetic : public IFunction
static FunctionOverloadResolverPtr
getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextConstPtr context)
{
bool first_is_date_or_datetime = isDateOrDateTime(type0);
bool second_is_date_or_datetime = isDateOrDateTime(type1);
bool first_is_date_or_datetime = isDate(type0) || isDateTime(type0) || isDateTime64(type0);
bool second_is_date_or_datetime = isDate(type1) || isDateTime(type1) || isDateTime64(type1);
/// Exactly one argument must be Date or DateTime
if (first_is_date_or_datetime == second_is_date_or_datetime)
@ -775,7 +775,7 @@ class FunctionBinaryArithmetic : public IFunction
ColumnsWithTypeAndName new_arguments = arguments;
/// Interval argument must be second.
if (WhichDataType(arguments[1].type).isDateOrDateTime())
if (isDate(arguments[1].type) || isDateTime(arguments[1].type) || isDateTime64(arguments[1].type))
std::swap(new_arguments[0], new_arguments[1]);
/// Change interval argument type to its representation
@ -990,7 +990,7 @@ public:
new_arguments[i].type = arguments[i];
/// Interval argument must be second.
if (WhichDataType(new_arguments[1].type).isDateOrDateTime())
if (isDate(new_arguments[1].type) || isDateTime(new_arguments[1].type) || isDateTime64(new_arguments[1].type))
std::swap(new_arguments[0], new_arguments[1]);
/// Change interval argument to its representation

View File

@ -35,7 +35,7 @@ public:
{
if (arguments.size() == 1)
{
if (!isDateOrDateTime(arguments[0].type))
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
throw Exception(
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
+ ". Should be a date or a date with time",
@ -43,7 +43,7 @@ public:
}
else if (arguments.size() == 2)
{
if (!isDateOrDateTime(arguments[0].type))
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
throw Exception(
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
+ ". Should be a date or a date with time",
@ -59,7 +59,7 @@ public:
}
else if (arguments.size() == 3)
{
if (!isDateOrDateTime(arguments[0].type))
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
throw Exception(
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
+ ". Should be a date or a date with time",

View File

@ -375,7 +375,7 @@ public:
if (arguments.size() == 2)
{
if (!isDateOrDateTime(arguments[0].type))
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() +
". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}

View File

@ -38,7 +38,7 @@ public:
{
if (arguments.size() == 1)
{
if (!isDateOrDateTime(arguments[0].type))
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
throw Exception(
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
+ ". Should be a date or a date with time",
@ -46,7 +46,7 @@ public:
}
else if (arguments.size() == 2)
{
if (!isDateOrDateTime(arguments[0].type))
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
throw Exception(
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
+ ". Should be a date or a date with time",

View File

@ -970,7 +970,9 @@ public:
WhichDataType which(arguments[0]);
if (!which.isStringOrFixedString() &&
!which.isDateOrDateTime() &&
!which.isDate() &&
!which.isDateTime() &&
!which.isDateTime64() &&
!which.isUInt() &&
!which.isFloat() &&
!which.isDecimal())

View File

@ -1086,7 +1086,7 @@ public:
if (!((both_represented_by_number && !has_date) /// Do not allow to compare date and number.
|| (left.isStringOrFixedString() || right.isStringOrFixedString()) /// Everything can be compared with string by conversion.
/// You can compare the date, datetime, or datatime64 and an enumeration with a constant string.
|| (left.isDateOrDateTime() && right.isDateOrDateTime() && left.idx == right.idx) /// only date vs date, or datetime vs datetime
|| ((left.isDate() || left.isDateTime() || left.isDateTime64()) && (right.isDate() || right.isDateTime() || right.isDateTime64()) && left.idx == right.idx) /// only date vs date, or datetime vs datetime
|| (left.isUUID() && right.isUUID())
|| (left.isEnum() && right.isEnum() && arguments[0]->getName() == arguments[1]->getName()) /// only equivalent enum type values can be compared against
|| (left_tuple && right_tuple && left_tuple->getElements().size() == right_tuple->getElements().size())
@ -1176,8 +1176,8 @@ public:
const bool left_is_string = isStringOrFixedString(which_left);
const bool right_is_string = isStringOrFixedString(which_right);
bool date_and_datetime = (which_left.idx != which_right.idx) &&
which_left.isDateOrDateTime() && which_right.isDateOrDateTime();
bool date_and_datetime = (which_left.idx != which_right.idx) && (which_left.isDate() || which_left.isDateTime() || which_left.isDateTime64())
&& (which_right.isDate() || which_right.isDateTime() || which_right.isDateTime64());
ColumnPtr res;
if (left_is_num && right_is_num && !date_and_datetime)

View File

@ -1890,7 +1890,7 @@ struct ToDateMonotonicity
static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right)
{
auto which = WhichDataType(type);
if (which.isDateOrDateTime() || which.isInt8() || which.isInt16() || which.isUInt8() || which.isUInt16())
if (which.isDate() || which.isDateTime() || which.isDateTime64() || which.isInt8() || which.isInt16() || which.isUInt8() || which.isUInt16())
return {true, true, true};
else if (
(which.isUInt() && ((left.isNull() || left.get<UInt64>() < 0xFFFF) && (right.isNull() || right.get<UInt64>() >= 0xFFFF)))
@ -2408,7 +2408,7 @@ private:
UInt32 scale = to_type->getScale();
WhichDataType which(type_index);
bool ok = which.isNativeInt() || which.isNativeUInt() || which.isDecimal() || which.isFloat() || which.isDateOrDateTime()
bool ok = which.isNativeInt() || which.isNativeUInt() || which.isDecimal() || which.isFloat() || which.isDate() || which.isDateTime() || which.isDateTime64()
|| which.isStringOrFixedString();
if (!ok)
{

View File

@ -217,7 +217,9 @@ FunctionArrayIntersect::CastArgumentsResult FunctionArrayIntersect::castColumns(
auto type_not_nullable_nested = removeNullable(type_nested);
const bool is_numeric_or_string = isNativeNumber(type_not_nullable_nested)
|| isDateOrDateTime(type_not_nullable_nested)
|| isDate(type_not_nullable_nested)
|| isDateTime(type_not_nullable_nested)
|| isDateTime64(type_not_nullable_nested)
|| isStringOrFixedString(type_not_nullable_nested);
DataTypePtr nullable_return_type;
@ -334,7 +336,7 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(
const auto & nested_init_type = typeid_cast<const DataTypeArray *>(removeNullable(initial_columns[i].type).get())->getNestedType();
const auto & nested_cast_type = typeid_cast<const DataTypeArray *>(removeNullable(columns[i].type).get())->getNestedType();
if (isInteger(nested_init_type) || isDateOrDateTime(nested_init_type))
if (isInteger(nested_init_type) || isDate(nested_init_type) || isDateTime(nested_init_type) || isDateTime64(nested_init_type))
{
/// Compare original and casted columns. It seem to be the easiest way.
auto overflow_mask = callFunctionNotEquals(

View File

@ -68,11 +68,11 @@ public:
throw Exception("First argument for function " + getName() + " (unit) must be String",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!isDateOrDateTime(arguments[1]))
if (!isDate(arguments[1]) && !isDateTime(arguments[1]) && !isDateTime64(arguments[1]))
throw Exception("Second argument for function " + getName() + " must be Date or DateTime",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!isDateOrDateTime(arguments[2]))
if (!isDate(arguments[2]) && !isDateTime(arguments[2]) && !isDateTime64(arguments[2]))
throw Exception("Third argument for function " + getName() + " must be Date or DateTime",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

View File

@ -62,7 +62,7 @@ public:
bool second_argument_is_date = false;
auto check_second_argument = [&] {
if (!isDateOrDateTime(arguments[1].type))
if (!isDate(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type))
throw Exception(
"Illegal type " + arguments[1].type->getName() + " of 2nd argument of function " + getName()
+ ". Should be a date or a date with time",

View File

@ -309,7 +309,7 @@ public:
"Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName()
+ " when arguments size is 1. Should be integer",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (arguments.size() > 1 && !(isInteger(arguments[0].type) || WhichDataType(arguments[0].type).isDateOrDateTime()))
if (arguments.size() > 1 && !(isInteger(arguments[0].type) || isDate(arguments[0].type) || isDateTime(arguments[0].type) || isDateTime64(arguments[0].type)))
throw Exception(
"Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName()
+ " when arguments size is 2 or 3. Should be a integer or a date with time",
@ -322,7 +322,7 @@ public:
"Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size())
+ ", should be 2 or 3",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!WhichDataType(arguments[0].type).isDateOrDateTime())
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
throw Exception(
"Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName()
+ ". Should be a date or a date with time",

View File

@ -259,7 +259,9 @@ private:
{
return type.isUInt() ||
type.isInt() ||
type.isDateOrDateTime() ||
type.isDate() ||
type.isDateTime() ||
type.isDateTime64() ||
type.isFloat() ||
type.isUUID() ||
type.isDecimal();

View File

@ -197,7 +197,7 @@ public:
bool first_argument_is_date = false;
auto check_first_argument = [&]
{
if (!isDateOrDateTime(arguments[0].type))
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
throw Exception(
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
+ ". Should be a date or a date with time",

View File

@ -1,4 +1,5 @@
#include <Common/Exception.h>
#include <Common/Throttler.h>
#include <IO/ReadBuffer.h>
#include <IO/WriteBuffer.h>
#include <IO/copyData.h>
@ -14,7 +15,7 @@ namespace ErrorCodes
namespace
{
void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t bytes, const std::atomic<int> * is_cancelled)
void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t bytes, const std::atomic<int> * is_cancelled, ThrottlerPtr throttler)
{
/// If read to the end of the buffer, eof() either fills the buffer with new data and moves the cursor to the beginning, or returns false.
while (bytes > 0 && !from.eof())
@ -27,13 +28,16 @@ void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t
to.write(from.position(), count);
from.position() += count;
bytes -= count;
if (throttler)
throttler->add(count);
}
if (check_bytes && bytes > 0)
throw Exception("Attempt to read after EOF.", ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF);
}
void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t bytes, std::function<void()> cancellation_hook)
void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t bytes, std::function<void()> cancellation_hook, ThrottlerPtr throttler)
{
/// If read to the end of the buffer, eof() either fills the buffer with new data and moves the cursor to the beginning, or returns false.
while (bytes > 0 && !from.eof())
@ -46,6 +50,9 @@ void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t
to.write(from.position(), count);
from.position() += count;
bytes -= count;
if (throttler)
throttler->add(count);
}
if (check_bytes && bytes > 0)
@ -56,32 +63,42 @@ void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t
void copyData(ReadBuffer & from, WriteBuffer & to)
{
copyDataImpl(from, to, false, std::numeric_limits<size_t>::max(), nullptr);
copyDataImpl(from, to, false, std::numeric_limits<size_t>::max(), nullptr, nullptr);
}
void copyData(ReadBuffer & from, WriteBuffer & to, const std::atomic<int> & is_cancelled)
{
copyDataImpl(from, to, false, std::numeric_limits<size_t>::max(), &is_cancelled);
copyDataImpl(from, to, false, std::numeric_limits<size_t>::max(), &is_cancelled, nullptr);
}
void copyData(ReadBuffer & from, WriteBuffer & to, std::function<void()> cancellation_hook)
{
copyDataImpl(from, to, false, std::numeric_limits<size_t>::max(), cancellation_hook);
copyDataImpl(from, to, false, std::numeric_limits<size_t>::max(), cancellation_hook, nullptr);
}
void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes)
{
copyDataImpl(from, to, true, bytes, nullptr);
copyDataImpl(from, to, true, bytes, nullptr, nullptr);
}
void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes, const std::atomic<int> & is_cancelled)
{
copyDataImpl(from, to, true, bytes, &is_cancelled);
copyDataImpl(from, to, true, bytes, &is_cancelled, nullptr);
}
void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes, std::function<void()> cancellation_hook)
{
copyDataImpl(from, to, true, bytes, cancellation_hook);
copyDataImpl(from, to, true, bytes, cancellation_hook, nullptr);
}
void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, const std::atomic<int> & is_cancelled, ThrottlerPtr throttler)
{
copyDataImpl(from, to, false, std::numeric_limits<size_t>::max(), &is_cancelled, throttler);
}
void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, size_t bytes, const std::atomic<int> & is_cancelled, ThrottlerPtr throttler)
{
copyDataImpl(from, to, true, bytes, &is_cancelled, throttler);
}
}

View File

@ -9,22 +9,26 @@ namespace DB
class ReadBuffer;
class WriteBuffer;
class Throttler;
using ThrottlerPtr = std::shared_ptr<Throttler>;
/** Copies data from ReadBuffer to WriteBuffer, all that is.
*/
/// Copies data from ReadBuffer to WriteBuffer, all that is.
void copyData(ReadBuffer & from, WriteBuffer & to);
/** Copies `bytes` bytes from ReadBuffer to WriteBuffer. If there are no `bytes` bytes, then throws an exception.
*/
/// Copies `bytes` bytes from ReadBuffer to WriteBuffer. If there are no `bytes` bytes, then throws an exception.
void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes);
/** The same, with the condition to cancel.
*/
/// The same, with the condition to cancel.
void copyData(ReadBuffer & from, WriteBuffer & to, const std::atomic<int> & is_cancelled);
void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes, const std::atomic<int> & is_cancelled);
void copyData(ReadBuffer & from, WriteBuffer & to, std::function<void()> cancellation_hook);
void copyData(ReadBuffer & from, WriteBuffer & to, size_t bytes, std::function<void()> cancellation_hook);
/// Same as above but also use throttler to limit maximum speed
void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, const std::atomic<int> & is_cancelled, ThrottlerPtr throttler);
void copyDataWithThrottler(ReadBuffer & from, WriteBuffer & to, size_t bytes, const std::atomic<int> & is_cancelled, ThrottlerPtr throttler);
}

View File

@ -11,6 +11,7 @@
#include <Common/setThreadName.h>
#include <Common/Stopwatch.h>
#include <Common/formatReadable.h>
#include <Common/Throttler.h>
#include <Common/thread_local_rng.h>
#include <Coordination/KeeperStorageDispatcher.h>
#include <Compression/ICompressionCodec.h>
@ -363,6 +364,10 @@ struct ContextSharedPart
mutable std::optional<BackgroundSchedulePool> schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables)
mutable std::optional<BackgroundSchedulePool> distributed_schedule_pool; /// A thread pool that can run different jobs in background (used for distributed sends)
mutable std::optional<BackgroundSchedulePool> message_broker_schedule_pool; /// A thread pool that can run different jobs in background (used for message brokers, like RabbitMQ and Kafka)
mutable ThrottlerPtr replicated_fetches_throttler; /// A server-wide throttler for replicated fetches
mutable ThrottlerPtr replicated_sends_throttler; /// A server-wide throttler for replicated sends
MultiVersion<Macros> macros; /// Substitutions extracted from config.
std::unique_ptr<DDLWorker> ddl_worker; /// Process ddl commands from zk.
/// Rules for selecting the compression settings, depending on the size of the part.
@ -1628,6 +1633,26 @@ BackgroundSchedulePool & Context::getMessageBrokerSchedulePool() const
return *shared->message_broker_schedule_pool;
}
ThrottlerPtr Context::getReplicatedFetchesThrottler() const
{
auto lock = getLock();
if (!shared->replicated_fetches_throttler)
shared->replicated_fetches_throttler = std::make_shared<Throttler>(
settings.max_replicated_fetches_network_bandwidth_for_server);
return shared->replicated_fetches_throttler;
}
ThrottlerPtr Context::getReplicatedSendsThrottler() const
{
auto lock = getLock();
if (!shared->replicated_sends_throttler)
shared->replicated_sends_throttler = std::make_shared<Throttler>(
settings.max_replicated_sends_network_bandwidth_for_server);
return shared->replicated_sends_throttler;
}
bool Context::hasDistributedDDL() const
{
return getConfigRef().has("distributed_ddl");

View File

@ -113,6 +113,9 @@ using VolumePtr = std::shared_ptr<IVolume>;
struct NamedSession;
struct BackgroundTaskSchedulingSettings;
class Throttler;
using ThrottlerPtr = std::shared_ptr<Throttler>;
class ZooKeeperMetadataTransaction;
using ZooKeeperMetadataTransactionPtr = std::shared_ptr<ZooKeeperMetadataTransaction>;
@ -657,6 +660,9 @@ public:
BackgroundSchedulePool & getMessageBrokerSchedulePool() const;
BackgroundSchedulePool & getDistributedSchedulePool() const;
ThrottlerPtr getReplicatedFetchesThrottler() const;
ThrottlerPtr getReplicatedSendsThrottler() const;
/// Has distributed_ddl configuration or not.
bool hasDistributedDDL() const;
void setDDLWorker(std::unique_ptr<DDLWorker> ddl_worker);

View File

@ -281,7 +281,7 @@ static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys)
if (which.isNullable())
throw Exception("LOGICAL ERROR: MySQL primary key must be not null, it is a bug.", ErrorCodes::LOGICAL_ERROR);
if (which.isDateOrDateTime())
if (which.isDate() || which.isDateTime() || which.isDateTime64())
{
/// In any case, date or datetime is always the best partitioning key
return makeASTFunction("toYYYYMM", std::make_shared<ASTIdentifier>(primary_key.name));

View File

@ -189,7 +189,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
return dynamic_cast<const IDataTypeEnum &>(type).castToValue(src);
}
if (which_type.isDateOrDateTime() && !which_type.isDateTime64() && src.getType() == Field::Types::UInt64)
if ((which_type.isDate() || which_type.isDateTime()) && src.getType() == Field::Types::UInt64)
{
/// We don't need any conversion UInt64 is under type of Date and DateTime
return src;
@ -202,7 +202,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
}
if (which_type.isDateTime64()
&& (which_from_type.isNativeInt() || which_from_type.isNativeUInt() || which_from_type.isDateOrDateTime()))
&& (which_from_type.isNativeInt() || which_from_type.isNativeUInt() || which_from_type.isDate() || which_from_type.isDateTime() || which_from_type.isDateTime64()))
{
const auto scale = static_cast<const DataTypeDateTime64 &>(type).getScale();
const auto decimal_value = DecimalUtils::decimalFromComponents<DateTime64>(src.reinterpret<Int64>(), 0, scale);

View File

@ -90,8 +90,11 @@ namespace DB
arrow::BinaryArray & chunk = static_cast<arrow::BinaryArray &>(*(arrow_column->chunk(chunk_i)));
const size_t chunk_length = chunk.length();
chars_t_size += chunk.value_offset(chunk_length - 1) + chunk.value_length(chunk_length - 1);
chars_t_size += chunk_length; /// additional space for null bytes
if (chunk_length > 0)
{
chars_t_size += chunk.value_offset(chunk_length - 1) + chunk.value_length(chunk_length - 1);
chars_t_size += chunk_length; /// additional space for null bytes
}
}
column_chars_t.reserve(chars_t_size);

View File

@ -117,7 +117,7 @@ bool RowInputFormatWithDiagnosticInfo::deserializeFieldAndPrintDiagnosticInfo(co
if (curr_position < prev_position)
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
if (isNativeNumber(type) || isDateOrDateTime(type))
if (isNativeNumber(type) || isDate(type) || isDateTime(type) || isDateTime64(type))
{
/// An empty string instead of a value.
if (curr_position == prev_position)

View File

@ -219,7 +219,7 @@ void MySQLHandler::finishHandshake(MySQLProtocol::ConnectionPhase::HandshakeResp
int ret = socket().receiveBytes(buf + pos, packet_size - pos);
if (ret == 0)
{
throw Exception("Cannot read all data. Bytes read: " + std::to_string(pos) + ". Bytes expected: 3.", ErrorCodes::CANNOT_READ_ALL_DATA);
throw Exception("Cannot read all data. Bytes read: " + std::to_string(pos) + ". Bytes expected: 3", ErrorCodes::CANNOT_READ_ALL_DATA);
}
pos += ret;
}

View File

@ -1,15 +1,13 @@
#include <Storages/MergeTree/ActiveDataPartSet.h>
#include <Common/Exception.h>
#include <common/logger_useful.h>
#include <algorithm>
#include <cassert>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
ActiveDataPartSet::ActiveDataPartSet(MergeTreeDataFormatVersion format_version_, const Strings & names)
: format_version(format_version_)
@ -18,8 +16,8 @@ ActiveDataPartSet::ActiveDataPartSet(MergeTreeDataFormatVersion format_version_,
add(name);
}
bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts)
/// FIXME replace warnings with logical errors
bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts, Poco::Logger * log)
{
/// TODO make it exception safe (out_replaced_parts->push_back(...) may throw)
auto part_info = MergeTreePartInfo::fromPartName(name, format_version);
@ -40,7 +38,11 @@ bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts)
if (!part_info.contains(it->first))
{
if (!part_info.isDisjoint(it->first))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} intersects previous part {}. It is a bug.", name, it->first.getPartName());
{
if (log)
LOG_ERROR(log, "Part {} intersects previous part {}. It is a bug.", name, it->first.getPartName());
assert(false);
}
++it;
break;
}
@ -56,15 +58,18 @@ bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts)
/// Let's go to the right.
while (it != part_info_to_name.end() && part_info.contains(it->first))
{
if (part_info == it->first)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected duplicate part {}. It is a bug.", name);
assert(part_info != it->first);
if (out_replaced_parts)
out_replaced_parts->push_back(it->second);
part_info_to_name.erase(it++);
}
if (it != part_info_to_name.end() && !part_info.isDisjoint(it->first))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} intersects next part {}. It is a bug.", name, it->first.getPartName());
{
if (log)
LOG_ERROR(log, "Part {} intersects next part {}. It is a bug.", name, it->first.getPartName());
assert(false);
}
part_info_to_name.emplace(part_info, name);
return true;

View File

@ -5,6 +5,10 @@
#include <map>
#include <vector>
namespace Poco
{
class Logger;
}
namespace DB
{
@ -46,7 +50,7 @@ public:
/// Returns true if the part was actually added. If out_replaced_parts != nullptr, it will contain
/// parts that were replaced from the set by the newly added part.
bool add(const String & name, Strings * out_replaced_parts = nullptr);
bool add(const String & name, Strings * out_replaced_parts = nullptr, Poco::Logger * log = nullptr);
bool remove(const MergeTreePartInfo & part_info)
{

View File

@ -116,8 +116,8 @@ try
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
CurrentMetrics::values[pool_config.tasks_metric]--;
tryLogCurrentException(__PRETTY_FUNCTION__);
scheduleTask(/* with_backoff = */ true);
}
});
@ -128,8 +128,8 @@ try
catch (...)
{
/// With our Pool settings scheduleOrThrowOnError shouldn't throw exceptions, but for safety catch added here
tryLogCurrentException(__PRETTY_FUNCTION__);
CurrentMetrics::values[pool_config.tasks_metric]--;
tryLogCurrentException(__PRETTY_FUNCTION__);
scheduleTask(/* with_backoff = */ true);
}
}

View File

@ -9,6 +9,7 @@
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
#include <Storages/MergeTree/MergedBlockOutputStream.h>
#include <Storages/MergeTree/ReplicatedFetchList.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Common/CurrentMetrics.h>
#include <Common/NetException.h>
#include <IO/createReadBufferFromFileBase.h>
@ -86,6 +87,10 @@ struct ReplicatedFetchReadCallback
}
Service::Service(StorageReplicatedMergeTree & data_) :
data(data_), log(&Poco::Logger::get(data.getLogName() + " (Replicated PartsService)")) {}
std::string Service::getId(const std::string & node_id) const
{
return getEndpointId(node_id);
@ -243,6 +248,8 @@ void Service::sendPartFromMemory(
NativeBlockOutputStream block_out(out, 0, metadata_snapshot->getSampleBlock());
part->checksums.write(out);
block_out.write(part_in_memory->block);
data.getSendsThrottler()->add(part_in_memory->block.bytes());
}
MergeTreeData::DataPart::Checksums Service::sendPartFromDisk(
@ -298,7 +305,7 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDisk(
auto file_in = disk->readFile(path);
HashingWriteBuffer hashing_out(out);
copyData(*file_in, hashing_out, blocker.getCounter());
copyDataWithThrottler(*file_in, hashing_out, blocker.getCounter(), data.getSendsThrottler());
if (blocker.isCancelled())
throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED);
@ -354,7 +361,7 @@ void Service::sendPartS3Metadata(const MergeTreeData::DataPartPtr & part, WriteB
auto file_in = createReadBufferFromFileBase(metadata_file, 0, 0, 0, nullptr, DBMS_DEFAULT_BUFFER_SIZE);
HashingWriteBuffer hashing_out(out);
copyData(*file_in, hashing_out, blocker.getCounter());
copyDataWithThrottler(*file_in, hashing_out, blocker.getCounter(), data.getSendsThrottler());
if (blocker.isCancelled())
throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED);
@ -388,6 +395,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
const String & user,
const String & password,
const String & interserver_scheme,
ThrottlerPtr throttler,
bool to_detached,
const String & tmp_prefix_,
std::optional<CurrentlySubmergingEmergingTagger> * tagger_ptr,
@ -514,7 +522,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
try
{
return downloadPartToS3(part_name, replica_path, to_detached, tmp_prefix_, std::move(disks_s3), in);
return downloadPartToS3(part_name, replica_path, to_detached, tmp_prefix_, std::move(disks_s3), in, throttler);
}
catch (const Exception & e)
{
@ -522,7 +530,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
throw;
/// Try again but without S3 copy
return fetchPart(metadata_snapshot, context, part_name, replica_path, host, port, timeouts,
user, password, interserver_scheme, to_detached, tmp_prefix_, nullptr, false);
user, password, interserver_scheme, throttler, to_detached, tmp_prefix_, nullptr, false);
}
}
@ -585,8 +593,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
MergeTreeData::DataPart::Checksums checksums;
return part_type == "InMemory"
? downloadPartToMemory(part_name, part_uuid, metadata_snapshot, context, std::move(reservation), in, projections)
: downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix_, sync, reservation->getDisk(), in, projections, checksums);
? downloadPartToMemory(part_name, part_uuid, metadata_snapshot, context, std::move(reservation), in, projections, throttler)
: downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix_, sync, reservation->getDisk(), in, projections, checksums, throttler);
}
MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
@ -596,7 +604,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
ContextPtr context,
ReservationPtr reservation,
PooledReadWriteBufferFromHTTP & in,
size_t projections)
size_t projections,
ThrottlerPtr throttler)
{
auto volume = std::make_shared<SingleDiskVolume>("volume_" + part_name, reservation->getDisk(), 0);
MergeTreeData::MutableDataPartPtr new_data_part =
@ -612,6 +621,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
NativeBlockInputStream block_in(in, 0);
auto block = block_in.read();
throttler->add(block.bytes());
MergeTreePartInfo new_part_info("all", 0, 0, 0);
MergeTreeData::MutableDataPartPtr new_projection_part =
@ -643,6 +653,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
NativeBlockInputStream block_in(in, 0);
auto block = block_in.read();
throttler->add(block.bytes());
new_data_part->uuid = part_uuid;
new_data_part->is_temp = true;
@ -666,7 +677,8 @@ void Fetcher::downloadBaseOrProjectionPartToDisk(
bool sync,
DiskPtr disk,
PooledReadWriteBufferFromHTTP & in,
MergeTreeData::DataPart::Checksums & checksums) const
MergeTreeData::DataPart::Checksums & checksums,
ThrottlerPtr throttler) const
{
size_t files;
readBinary(files, in);
@ -689,7 +701,7 @@ void Fetcher::downloadBaseOrProjectionPartToDisk(
auto file_out = disk->writeFile(fs::path(part_download_path) / file_name);
HashingWriteBuffer hashing_out(*file_out);
copyData(in, hashing_out, file_size, blocker.getCounter());
copyDataWithThrottler(in, hashing_out, file_size, blocker.getCounter(), throttler);
if (blocker.isCancelled())
{
@ -726,7 +738,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk(
DiskPtr disk,
PooledReadWriteBufferFromHTTP & in,
size_t projections,
MergeTreeData::DataPart::Checksums & checksums)
MergeTreeData::DataPart::Checksums & checksums,
ThrottlerPtr throttler)
{
static const String TMP_PREFIX = "tmp_fetch_";
String tmp_prefix = tmp_prefix_.empty() ? TMP_PREFIX : tmp_prefix_;
@ -763,13 +776,13 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk(
MergeTreeData::DataPart::Checksums projection_checksum;
disk->createDirectories(part_download_path + projection_name + ".proj/");
downloadBaseOrProjectionPartToDisk(
replica_path, part_download_path + projection_name + ".proj/", sync, disk, in, projection_checksum);
replica_path, part_download_path + projection_name + ".proj/", sync, disk, in, projection_checksum, throttler);
checksums.addFile(
projection_name + ".proj", projection_checksum.getTotalSizeOnDisk(), projection_checksum.getTotalChecksumUInt128());
}
// Download the base part
downloadBaseOrProjectionPartToDisk(replica_path, part_download_path, sync, disk, in, checksums);
downloadBaseOrProjectionPartToDisk(replica_path, part_download_path, sync, disk, in, checksums, throttler);
assertEOF(in);
auto volume = std::make_shared<SingleDiskVolume>("volume_" + part_name, disk, 0);
@ -787,8 +800,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToS3(
bool to_detached,
const String & tmp_prefix_,
const Disks & disks_s3,
PooledReadWriteBufferFromHTTP & in
)
PooledReadWriteBufferFromHTTP & in,
ThrottlerPtr throttler)
{
if (disks_s3.empty())
throw Exception("No S3 disks anymore", ErrorCodes::LOGICAL_ERROR);
@ -841,7 +854,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToS3(
HashingWriteBuffer hashing_out(*file_out);
copyData(in, hashing_out, file_size, blocker.getCounter());
copyDataWithThrottler(in, hashing_out, file_size, blocker.getCounter(), throttler);
if (blocker.isCancelled())
{

View File

@ -7,6 +7,7 @@
#include <IO/copyData.h>
#include <IO/ConnectionTimeouts.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <Common/Throttler.h>
namespace zkutil
@ -18,15 +19,17 @@ namespace zkutil
namespace DB
{
class StorageReplicatedMergeTree;
namespace DataPartsExchange
{
/** Service for sending parts from the table *MergeTree.
/** Service for sending parts from the table *ReplicatedMergeTree.
*/
class Service final : public InterserverIOEndpoint
{
public:
explicit Service(MergeTreeData & data_) : data(data_), log(&Poco::Logger::get(data.getLogName() + " (Replicated PartsService)")) {}
explicit Service(StorageReplicatedMergeTree & data_);
Service(const Service &) = delete;
Service & operator=(const Service &) = delete;
@ -51,7 +54,7 @@ private:
/// StorageReplicatedMergeTree::shutdown() waits for all parts exchange handlers to finish,
/// so Service will never access dangling reference to storage
MergeTreeData & data;
StorageReplicatedMergeTree & data;
Poco::Logger * log;
};
@ -74,6 +77,7 @@ public:
const String & user,
const String & password,
const String & interserver_scheme,
ThrottlerPtr throttler,
bool to_detached = false,
const String & tmp_prefix_ = "",
std::optional<CurrentlySubmergingEmergingTagger> * tagger_ptr = nullptr,
@ -90,7 +94,9 @@ private:
bool sync,
DiskPtr disk,
PooledReadWriteBufferFromHTTP & in,
MergeTreeData::DataPart::Checksums & checksums) const;
MergeTreeData::DataPart::Checksums & checksums,
ThrottlerPtr throttler) const;
MergeTreeData::MutableDataPartPtr downloadPartToDisk(
const String & part_name,
@ -101,7 +107,8 @@ private:
DiskPtr disk,
PooledReadWriteBufferFromHTTP & in,
size_t projections,
MergeTreeData::DataPart::Checksums & checksums);
MergeTreeData::DataPart::Checksums & checksums,
ThrottlerPtr throttler);
MergeTreeData::MutableDataPartPtr downloadPartToMemory(
const String & part_name,
@ -110,7 +117,8 @@ private:
ContextPtr context,
ReservationPtr reservation,
PooledReadWriteBufferFromHTTP & in,
size_t projections);
size_t projections,
ThrottlerPtr throttler);
MergeTreeData::MutableDataPartPtr downloadPartToS3(
const String & part_name,
@ -118,7 +126,8 @@ private:
bool to_detached,
const String & tmp_prefix_,
const Disks & disks_s3,
PooledReadWriteBufferFromHTTP & in);
PooledReadWriteBufferFromHTTP & in,
ThrottlerPtr throttler);
MergeTreeData & data;
Poco::Logger * log;

View File

@ -4908,7 +4908,11 @@ void MergeTreeData::removeQueryId(const String & query_id) const
{
std::lock_guard lock(query_id_set_mutex);
if (query_id_set.find(query_id) == query_id_set.end())
{
/// Do not throw exception, because this method is used in destructor.
LOG_WARNING(log, "We have query_id removed but it's not recorded. This is a bug");
assert(false);
}
else
query_id_set.erase(query_id);
}
@ -5088,7 +5092,10 @@ CurrentlySubmergingEmergingTagger::~CurrentlySubmergingEmergingTagger()
for (const auto & part : submerging_parts)
{
if (!storage.currently_submerging_big_parts.count(part))
LOG_WARNING(log, "currently_submerging_big_parts doesn't contain part {} to erase. This is a bug", part->name);
{
LOG_ERROR(log, "currently_submerging_big_parts doesn't contain part {} to erase. This is a bug", part->name);
assert(false);
}
else
storage.currently_submerging_big_parts.erase(part);
}

View File

@ -85,7 +85,7 @@ static void assertIndexColumnsType(const Block & header)
WhichDataType which(actual_type);
if (!which.isUInt() && !which.isInt() && !which.isString() && !which.isFixedString() && !which.isFloat() &&
!which.isDateOrDateTime() && !which.isEnum() && !which.isUUID())
!which.isDate() && !which.isDateTime() && !which.isDateTime64() && !which.isEnum() && !which.isUUID())
throw Exception("Unexpected type " + type->getName() + " of bloom filter index.",
ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -71,6 +71,13 @@ struct MergeTreePartInfo
|| max_block < rhs.min_block;
}
bool isFakeDropRangePart() const
{
/// Another max level was previously used for REPLACE/MOVE PARTITION
auto another_max_level = std::numeric_limits<decltype(level)>::max();
return level == MergeTreePartInfo::MAX_LEVEL || level == another_max_level;
}
String getPartName() const;
String getPartNameV0(DayNum left_date, DayNum right_date) const;
UInt64 getBlocksCount() const

View File

@ -92,6 +92,8 @@ struct Settings;
M(Bool, replicated_can_become_leader, true, "If true, Replicated tables replicas on this node will try to acquire leadership.", 0) \
M(Seconds, zookeeper_session_expiration_check_period, 60, "ZooKeeper session expiration check period, in seconds.", 0) \
M(Bool, detach_old_local_parts_when_cloning_replica, 1, "Do not remove old local parts when repairing lost replica.", 0) \
M(UInt64, max_replicated_fetches_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \
M(UInt64, max_replicated_sends_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \
\
/** Check delay of replicas settings. */ \
M(UInt64, min_relative_delay_to_measure, 120, "Calculate relative replica delay only if absolute delay is not less that this value.", 0) \

View File

@ -413,6 +413,24 @@ ReplicatedMergeTreeLogEntry::Ptr ReplicatedMergeTreeLogEntry::parse(const String
return res;
}
std::optional<String> ReplicatedMergeTreeLogEntryData::getDropRange(MergeTreeDataFormatVersion format_version) const
{
if (type == DROP_RANGE)
return new_part_name;
if (type == REPLACE_RANGE)
{
auto drop_range_info = MergeTreePartInfo::fromPartName(replace_range_entry->drop_range_part_name, format_version);
if (!ReplaceRangeEntry::isMovePartitionOrAttachFrom(drop_range_info))
{
/// It's REPLACE, not MOVE or ATTACH, so drop range is real
return replace_range_entry->drop_range_part_name;
}
}
return {};
}
Strings ReplicatedMergeTreeLogEntryData::getVirtualPartNames(MergeTreeDataFormatVersion format_version) const
{
/// Doesn't produce any part
@ -431,11 +449,8 @@ Strings ReplicatedMergeTreeLogEntryData::getVirtualPartNames(MergeTreeDataFormat
{
Strings res = replace_range_entry->new_part_names;
auto drop_range_info = MergeTreePartInfo::fromPartName(replace_range_entry->drop_range_part_name, format_version);
if (!ReplaceRangeEntry::isMovePartitionOrAttachFrom(drop_range_info))
{
/// It's REPLACE, not MOVE or ATTACH, so drop range is real
res.emplace_back(replace_range_entry->drop_range_part_name);
}
if (auto drop_range = getDropRange(format_version))
res.emplace_back(*drop_range);
return res;
}

View File

@ -152,6 +152,9 @@ struct ReplicatedMergeTreeLogEntryData
return res;
}
/// Returns fake part for drop range (for DROP_RANGE and REPLACE_RANGE)
std::optional<String> getDropRange(MergeTreeDataFormatVersion format_version) const;
/// Access under queue_mutex, see ReplicatedMergeTreeQueue.
bool currently_executing = false; /// Whether the action is executing now.
bool removed_by_other_entry = false;

View File

@ -18,6 +18,7 @@ namespace DB
namespace ErrorCodes
{
extern const int TABLE_DIFFERS_TOO_MUCH;
extern const int LOGICAL_ERROR;
}
static const auto PART_CHECK_ERROR_SLEEP_MS = 5 * 1000;
@ -367,8 +368,8 @@ void ReplicatedMergeTreePartCheckThread::run()
{
if (!parts_set.empty())
{
LOG_ERROR(log, "Non-empty parts_set with empty parts_queue. This is a bug.");
parts_set.clear();
throw Exception(ErrorCodes::LOGICAL_ERROR, "Non-empty parts_set with empty parts_queue. This is a bug.");
}
}
else
@ -401,7 +402,7 @@ void ReplicatedMergeTreePartCheckThread::run()
if (parts_queue.empty())
{
LOG_ERROR(log, "Someone erased checking part from parts_queue. This is a bug.");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Someone erased checking part from parts_queue. This is a bug.");
}
else
{

View File

@ -34,24 +34,29 @@ ReplicatedMergeTreeQueue::ReplicatedMergeTreeQueue(StorageReplicatedMergeTree &
}
void ReplicatedMergeTreeQueue::clear()
{
auto locks = lockQueue();
assert(future_parts.empty());
current_parts.clear();
virtual_parts.clear();
queue.clear();
inserts_by_time.clear();
mutations_by_znode.clear();
mutations_by_partition.clear();
mutation_pointer.clear();
}
void ReplicatedMergeTreeQueue::initialize(const MergeTreeData::DataParts & parts)
{
addVirtualParts(parts);
}
void ReplicatedMergeTreeQueue::addVirtualParts(const MergeTreeData::DataParts & parts)
{
std::lock_guard lock(state_mutex);
for (const auto & part : parts)
{
current_parts.add(part->name);
virtual_parts.add(part->name);
current_parts.add(part->name, nullptr, log);
virtual_parts.add(part->name, nullptr, log);
}
}
bool ReplicatedMergeTreeQueue::isVirtualPart(const MergeTreeData::DataPartPtr & data_part) const
{
std::lock_guard lock(state_mutex);
@ -74,9 +79,6 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper)
/// Reset batch size on initialization to recover from possible errors of too large batch size.
current_multi_batch_size = 1;
String log_pointer_str = zookeeper->get(fs::path(replica_path) / "log_pointer");
log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str);
std::unordered_set<String> already_loaded_paths;
{
std::lock_guard lock(state_mutex);
@ -134,7 +136,7 @@ void ReplicatedMergeTreeQueue::insertUnlocked(
{
for (const String & virtual_part_name : entry->getVirtualPartNames(format_version))
{
virtual_parts.add(virtual_part_name);
virtual_parts.add(virtual_part_name, nullptr, log);
addPartToMutations(virtual_part_name);
}
@ -221,23 +223,17 @@ void ReplicatedMergeTreeQueue::updateStateOnQueueEntryRemoval(
for (const String & virtual_part_name : entry->getVirtualPartNames(format_version))
{
current_parts.add(virtual_part_name);
current_parts.add(virtual_part_name, nullptr, log);
/// These parts are already covered by newer part, we don't have to
/// mutate it.
removeCoveredPartsFromMutations(virtual_part_name, /*remove_part = */ false, /*remove_covered_parts = */ true);
}
String drop_range_part_name;
if (entry->type == LogEntry::DROP_RANGE)
drop_range_part_name = entry->new_part_name;
else if (entry->type == LogEntry::REPLACE_RANGE)
drop_range_part_name = entry->replace_range_entry->drop_range_part_name;
if (!drop_range_part_name.empty())
if (auto drop_range_part_name = entry->getDropRange(format_version))
{
current_parts.remove(drop_range_part_name);
virtual_parts.remove(drop_range_part_name);
current_parts.remove(*drop_range_part_name);
virtual_parts.remove(*drop_range_part_name);
}
if (entry->type == LogEntry::ALTER_METADATA)
@ -302,9 +298,7 @@ void ReplicatedMergeTreeQueue::addPartToMutations(const String & part_name)
auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version);
/// Do not add special virtual parts to parts_to_do
auto max_level = MergeTreePartInfo::MAX_LEVEL; /// DROP/DETACH PARTITION
auto another_max_level = std::numeric_limits<decltype(part_info.level)>::max(); /// REPLACE/MOVE PARTITION
if (part_info.level == max_level || part_info.level == another_max_level)
if (part_info.isFakeDropRangePart())
return;
auto in_partition = mutations_by_partition.find(part_info.partition_id);
@ -344,7 +338,9 @@ void ReplicatedMergeTreeQueue::updateTimesInZooKeeper(
auto code = zookeeper->tryMulti(ops, responses);
if (code != Coordination::Error::ZOK)
LOG_ERROR(log, "Couldn't set value of nodes for insert times ({}/min_unprocessed_insert_time, max_processed_insert_time): {}. This shouldn't happen often.", replica_path, Coordination::errorMessage(code));
LOG_ERROR(log, "Couldn't set value of nodes for insert times "
"({}/min_unprocessed_insert_time, max_processed_insert_time): {}. "
"This shouldn't happen often.", replica_path, Coordination::errorMessage(code));
}
}
@ -392,7 +388,8 @@ void ReplicatedMergeTreeQueue::removeProcessedEntry(zkutil::ZooKeeperPtr zookeep
}
if (!found && need_remove_from_zk)
throw Exception("Can't find " + entry->znode_name + " in the memory queue. It is a bug", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't find {} in the memory queue. It is a bug. Entry: {}",
entry->znode_name, entry->toString());
notifySubscribers(queue_size);
@ -434,7 +431,7 @@ bool ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, const Stri
{
auto part_in_current_parts = current_parts.getContainingPart(source_part);
if (part_in_current_parts == source_part)
virtual_parts.add(source_part);
virtual_parts.add(source_part, nullptr, log);
}
}
@ -462,8 +459,9 @@ bool ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, const Stri
}
bool ReplicatedMergeTreeQueue::removeFromVirtualParts(const MergeTreePartInfo & part_info)
bool ReplicatedMergeTreeQueue::removeFailedQuorumPart(const MergeTreePartInfo & part_info)
{
assert(part_info.level == 0);
std::lock_guard lock(state_mutex);
return virtual_parts.remove(part_info);
}
@ -587,8 +585,6 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper
{
std::lock_guard state_lock(state_mutex);
log_pointer = last_entry_index + 1;
for (size_t copied_entry_idx = 0, num_copied_entries = copied_entries.size(); copied_entry_idx < num_copied_entries; ++copied_entry_idx)
{
String path_created = dynamic_cast<const Coordination::CreateResponse &>(*responses[copied_entry_idx]).path_created;
@ -758,9 +754,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C
/// Such parts do not exist and will never appear, so we should not add virtual parts to parts_to_do list.
/// Fortunately, it's easy to distinguish virtual parts from normal parts by part level.
/// See StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(...)
auto max_level = MergeTreePartInfo::MAX_LEVEL; /// DROP/DETACH PARTITION
auto another_max_level = std::numeric_limits<decltype(part_info.level)>::max(); /// REPLACE/MOVE PARTITION
if (part_info.level == max_level || part_info.level == another_max_level)
if (part_info.isFakeDropRangePart())
continue;
auto it = entry->block_numbers.find(part_info.partition_id);
@ -941,9 +935,6 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange(
if ((*it)->currently_executing)
to_wait.push_back(*it);
auto code = zookeeper->tryRemove(fs::path(replica_path) / "queue" / (*it)->znode_name);
/// FIXME it's probably unsafe to remove entries non-atomically
/// when this method called directly from alter query (not from replication queue task),
/// because entries will be lost if ALTER fails.
if (code != Coordination::Error::ZOK)
LOG_INFO(log, "Couldn't remove {}: {}", (fs::path(replica_path) / "queue" / (*it)->znode_name).string(), Coordination::errorMessage(code));
@ -1259,7 +1250,9 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::CurrentlyExecuting(const Replicate
for (const String & new_part_name : entry->getBlockingPartNames(queue.format_version))
{
if (!queue.future_parts.emplace(new_part_name, entry).second)
throw Exception("Tagging already tagged future part " + new_part_name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Tagging already tagged future part {}. This is a bug. "
"It happened on attempt to execute {}: {}",
new_part_name, entry->znode_name, entry->toString());
}
}
@ -1277,7 +1270,9 @@ void ReplicatedMergeTreeQueue::CurrentlyExecuting::setActualPartName(ReplicatedM
return;
if (!queue.future_parts.emplace(entry.actual_new_part_name, entry.shared_from_this()).second)
throw Exception("Attaching already existing future part " + entry.actual_new_part_name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attaching already existing future part {}. This is a bug. "
"It happened on attempt to execute {}: {}",
entry.actual_new_part_name, entry.znode_name, entry.toString());
}
@ -1296,13 +1291,19 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting()
for (const String & new_part_name : entry->getBlockingPartNames(queue.format_version))
{
if (!queue.future_parts.erase(new_part_name))
{
LOG_ERROR(queue.log, "Untagging already untagged future part {}. This is a bug.", new_part_name);
assert(false);
}
}
if (!entry->actual_new_part_name.empty())
{
if (entry->actual_new_part_name != entry->new_part_name && !queue.future_parts.erase(entry->actual_new_part_name))
{
LOG_ERROR(queue.log, "Untagging already untagged future part {}. This is a bug.", entry->actual_new_part_name);
assert(false);
}
entry->actual_new_part_name.clear();
}

View File

@ -92,9 +92,6 @@ private:
using FuturePartsSet = std::map<String, LogEntryPtr>;
FuturePartsSet future_parts;
/// Index of the first log entry that we didn't see yet.
Int64 log_pointer = 0;
/// Avoid parallel execution of queue enties, which may remove other entries from the queue.
bool currently_executing_drop_or_replace_range = false;
@ -183,9 +180,6 @@ private:
/// Ensures that only one thread is simultaneously updating mutations.
std::mutex update_mutations_mutex;
/// Put a set of (already existing) parts in virtual_parts.
void addVirtualParts(const MergeTreeData::DataParts & parts);
/// Insert new entry from log into queue
void insertUnlocked(
const LogEntryPtr & entry, std::optional<time_t> & min_unprocessed_insert_time_changed,
@ -275,7 +269,10 @@ public:
ReplicatedMergeTreeQueue(StorageReplicatedMergeTree & storage_, ReplicatedMergeTreeMergeStrategyPicker & merge_strategy_picker_);
~ReplicatedMergeTreeQueue();
/// Clears queue state
void clear();
/// Put a set of (already existing) parts in virtual_parts.
void initialize(const MergeTreeData::DataParts & parts);
/** Inserts an action to the end of the queue.
@ -295,7 +292,7 @@ public:
*/
bool load(zkutil::ZooKeeperPtr zookeeper);
bool removeFromVirtualParts(const MergeTreePartInfo & part_info);
bool removeFailedQuorumPart(const MergeTreePartInfo & part_info);
/** Copy the new entries from the shared log to the queue of this replica. Set the log_pointer to the appropriate value.
* If watch_callback is not empty, will call it when new entries appear in the log.

View File

@ -174,6 +174,9 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup()
storage.partial_shutdown_called = false;
storage.partial_shutdown_event.reset();
/// Start queue processing
storage.background_executor.start();
storage.queue_updating_task->activateAndSchedule();
storage.mutations_updating_task->activateAndSchedule();
storage.mutations_finalizing_task->activateAndSchedule();
@ -227,7 +230,7 @@ void ReplicatedMergeTreeRestartingThread::removeFailedQuorumParts()
{
LOG_DEBUG(log, "Found part {} with failed quorum. Moving to detached. This shouldn't happen often.", part_name);
storage.forgetPartAndMoveToDetached(part, "noquorum");
storage.queue.removeFromVirtualParts(part->info);
storage.queue.removeFailedQuorumPart(part->info);
}
}
}
@ -352,6 +355,9 @@ void ReplicatedMergeTreeRestartingThread::partialShutdown()
storage.cleanup_thread.stop();
storage.part_check_thread.stop();
/// Stop queue processing
storage.background_executor.finish();
LOG_TRACE(log, "Threads finished");
}

View File

@ -290,6 +290,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
, part_moves_between_shards_orchestrator(*this)
, allow_renaming(allow_renaming_)
, replicated_fetches_pool_size(getContext()->getSettingsRef().background_fetches_pool_size)
, replicated_fetches_throttler(std::make_shared<Throttler>(getSettings()->max_replicated_fetches_network_bandwidth, getContext()->getReplicatedFetchesThrottler()))
, replicated_sends_throttler(std::make_shared<Throttler>(getSettings()->max_replicated_sends_network_bandwidth, getContext()->getReplicatedSendsThrottler()))
{
queue_updating_task = getContext()->getSchedulePool().createTask(
getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::queueUpdatingTask)", [this]{ queueUpdatingTask(); });
@ -2048,7 +2050,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
if (code == Coordination::Error::ZOK)
{
LOG_DEBUG(log, "Marked quorum for part {} as failed.", entry.new_part_name);
queue.removeFromVirtualParts(part_info);
queue.removeFailedQuorumPart(part_info);
return true;
}
else if (code == Coordination::Error::ZBADVERSION || code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS)
@ -2061,7 +2063,10 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
}
else
{
LOG_WARNING(log, "No active replica has part {}, but that part needs quorum and /quorum/status contains entry about another part {}. It means that part was successfully written to {} replicas, but then all of them goes offline. Or it is a bug.", entry.new_part_name, quorum_entry.part_name, entry.quorum);
LOG_WARNING(log, "No active replica has part {}, "
"but that part needs quorum and /quorum/status contains entry about another part {}. "
"It means that part was successfully written to {} replicas, but then all of them goes offline. "
"Or it is a bug.", entry.new_part_name, quorum_entry.part_name, entry.quorum);
}
}
}
@ -2507,7 +2512,8 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
part_desc->res_part = fetcher.fetchPart(
metadata_snapshot, getContext(), part_desc->found_new_part_name, source_replica_path,
address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, false, TMP_PREFIX + "fetch_");
address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(),
interserver_scheme, replicated_fetches_throttler, false, TMP_PREFIX + "fetch_");
/// TODO: check columns_version of fetched part
@ -2624,7 +2630,8 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr
return fetcher.fetchPart(
metadata_snapshot, getContext(), entry.new_part_name, source_replica_path,
address.host, address.replication_port,
timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, true);
timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme,
replicated_fetches_throttler, true);
};
part = get_part();
@ -2749,7 +2756,6 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo
{
if (active_parts_set.getContainingPart(part).empty())
{
queue.remove(zookeeper, part);
parts_to_remove_from_zk.emplace_back(part);
LOG_WARNING(log, "Source replica does not have part {}. Removing it from ZooKeeper.", part);
}
@ -2994,6 +3000,7 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke
/// Clear obsolete queue that we no longer need.
zookeeper->removeChildren(fs::path(replica_path) / "queue");
queue.clear();
/// Will do repair from the selected replica.
cloneReplica(source_replica, source_is_lost_stat, zookeeper);
@ -3163,6 +3170,13 @@ bool StorageReplicatedMergeTree::canExecuteFetch(const ReplicatedMergeTreeLogEnt
return false;
}
if (replicated_fetches_throttler->isThrottling())
{
disable_reason = fmt::format("Not executing fetch of part {} because fetches have already throttled by network settings "
"<max_replicated_fetches_network_bandwidth> or <max_replicated_fetches_network_bandwidth_for_server>.", entry.new_part_name);
return false;
}
return true;
}
@ -4031,6 +4045,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora
credentials->getUser(),
credentials->getPassword(),
interserver_scheme,
replicated_fetches_throttler,
to_detached,
"",
&tagger_ptr,
@ -4180,7 +4195,8 @@ bool StorageReplicatedMergeTree::fetchExistsPart(const String & part_name, const
return fetcher.fetchPart(
metadata_snapshot, getContext(), part_name, source_replica_path,
address.host, address.replication_port,
timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, false, "", nullptr, true,
timeouts, credentials->getUser(), credentials->getPassword(),
interserver_scheme, replicated_fetches_throttler, false, "", nullptr, true,
replaced_disk);
};

View File

@ -26,6 +26,7 @@
#include <Interpreters/PartLog.h>
#include <Common/randomSeed.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/Throttler.h>
#include <Core/BackgroundSchedulePool.h>
#include <Processors/Pipe.h>
#include <Storages/MergeTree/BackgroundJobsExecutor.h>
@ -239,6 +240,18 @@ public:
/// Get best replica having this partition on S3
String getSharedDataReplica(const IMergeTreeDataPart & part) const;
/// Get throttler for replicated fetches
ThrottlerPtr getFetchesThrottler() const
{
return replicated_fetches_throttler;
}
/// Get throttler for replicated sends
ThrottlerPtr getSendsThrottler() const
{
return replicated_sends_throttler;
}
private:
/// Get a sequential consistent view of current parts.
ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock getMaxAddedBlocks() const;
@ -363,6 +376,11 @@ private:
const size_t replicated_fetches_pool_size;
/// Throttlers used in DataPartsExchange to lower maximum fetch/sends
/// speed.
ThrottlerPtr replicated_fetches_throttler;
ThrottlerPtr replicated_sends_throttler;
template <class Func>
void foreachCommittedParts(Func && func, bool select_sequential_consistency) const;

View File

@ -11,10 +11,13 @@ import json
import csv
MAX_RETRY = 2
MAX_RETRY = 3
NUM_WORKERS = 5
SLEEP_BETWEEN_RETRIES = 5
PARALLEL_GROUP_SIZE = 100
CLICKHOUSE_BINARY_PATH = "/usr/bin/clickhouse"
CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH = "/usr/bin/clickhouse-odbc-bridge"
DOCKERD_LOGS_PATH = "/ClickHouse/tests/integration/dockerd.log"
CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH = "/usr/bin/clickhouse-library-bridge"
TRIES_COUNT = 10
@ -50,6 +53,11 @@ def filter_existing_tests(tests_to_run, repo_path):
def _get_deselect_option(tests):
return ' '.join(['--deselect {}'.format(t) for t in tests])
# https://stackoverflow.com/questions/312443/how-do-you-split-a-list-into-evenly-sized-chunks
def chunks(lst, n):
"""Yield successive n-sized chunks from lst."""
for i in range(0, len(lst), n):
yield lst[i:i + n]
def parse_test_results_output(fname):
read = False
@ -82,7 +90,7 @@ def get_counters(output):
else:
logging.info("Strange line %s", line)
else:
logging.info("Strange line %s")
logging.info("Strange line %s", line)
return {k: list(v) for k, v in counters.items()}
@ -253,7 +261,7 @@ class ClickhouseIntegrationTestsRunner:
def _get_all_tests(self, repo_path):
image_cmd = self._get_runner_image_cmd(repo_path)
cmd = "cd {}/tests/integration && ./runner {} ' --setup-plan' | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' > all_tests.txt".format(repo_path, image_cmd)
cmd = "cd {}/tests/integration && ./runner --tmpfs {} ' --setup-plan' | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' | sort -u > all_tests.txt".format(repo_path, image_cmd)
logging.info("Getting all tests with cmd '%s'", cmd)
subprocess.check_call(cmd, shell=True) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL
@ -267,6 +275,16 @@ class ClickhouseIntegrationTestsRunner:
all_tests.append(line.strip())
return list(sorted(all_tests))
def _get_parallel_tests_skip_list(self, repo_path):
skip_list_file_path = "{}/tests/integration/parallel_skip.json".format(repo_path)
if not os.path.isfile(skip_list_file_path) or os.path.getsize(skip_list_file_path) == 0:
raise Exception("There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format(skip_list_file_path))
skip_list_tests = []
with open(skip_list_file_path, "r") as skip_list_file:
skip_list_tests = json.load(skip_list_file)
return list(sorted(skip_list_tests))
def group_test_by_file(self, tests):
result = {}
for test in tests:
@ -286,7 +304,6 @@ class ClickhouseIntegrationTestsRunner:
if test in main_counters["ERROR"]:
main_counters["ERROR"].remove(test)
is_flaky = True
if is_flaky:
main_counters["FLAKY"].append(test)
else:
@ -319,7 +336,7 @@ class ClickhouseIntegrationTestsRunner:
logging.info("Cannot run with custom docker image version :(")
return image_cmd
def run_test_group(self, repo_path, test_group, tests_in_group, num_tries):
def run_test_group(self, repo_path, test_group, tests_in_group, num_tries, num_workers):
counters = {
"ERROR": [],
"PASSED": [],
@ -359,8 +376,9 @@ class ClickhouseIntegrationTestsRunner:
test_names.add(test_name)
test_cmd = ' '.join([test for test in sorted(test_names)])
cmd = "cd {}/tests/integration && ./runner {} '-ss {} -rfEp --color=no --durations=0 {}' | tee {}".format(
repo_path, image_cmd, test_cmd, _get_deselect_option(self.should_skip_tests()), output_path)
parallel_cmd = " --parallel {} ".format(num_workers) if num_workers > 0 else ""
cmd = "cd {}/tests/integration && ./runner --tmpfs {} -t {} {} '-ss -rfEp --color=no --durations=0 {}' | tee {}".format(
repo_path, image_cmd, test_cmd, parallel_cmd, _get_deselect_option(self.should_skip_tests()), output_path)
with open(log_path, 'w') as log:
logging.info("Executing cmd: %s", cmd)
@ -412,7 +430,7 @@ class ClickhouseIntegrationTestsRunner:
for i in range(TRIES_COUNT):
final_retry += 1
logging.info("Running tests for the %s time", i)
counters, tests_times, log_paths = self.run_test_group(repo_path, "flaky", tests_to_run, 1)
counters, tests_times, log_paths = self.run_test_group(repo_path, "flaky", tests_to_run, 1, 1)
logs += log_paths
if counters["FAILED"]:
logging.info("Found failed tests: %s", ' '.join(counters["FAILED"]))
@ -439,6 +457,10 @@ class ClickhouseIntegrationTestsRunner:
self._compress_logs("{}/tests/integration".format(repo_path), test_logs)
logging.info("Compression finished")
result_path_dockerd_logs = os.path.join(str(self.path()), "dockerd.log")
if os.path.exists(result_path_dockerd_logs):
shutil.copy(DOCKERD_LOGS_PATH, result_path_dockerd_logs)
test_result = []
for state in ("ERROR", "FAILED", "PASSED", "SKIPPED", "FLAKY"):
if state == "PASSED":
@ -459,8 +481,19 @@ class ClickhouseIntegrationTestsRunner:
self._install_clickhouse(build_path)
logging.info("Dump iptables before run %s", subprocess.check_output("iptables -L", shell=True))
all_tests = self._get_all_tests(repo_path)
parallel_skip_tests = self._get_parallel_tests_skip_list(repo_path)
logging.info("Found %s tests first 3 %s", len(all_tests), ' '.join(all_tests[:3]))
grouped_tests = self.group_test_by_file(all_tests)
filtered_sequential_tests = list(filter(lambda test: test in all_tests, parallel_skip_tests))
filtered_parallel_tests = list(filter(lambda test: test not in parallel_skip_tests, all_tests))
not_found_tests = list(filter(lambda test: test not in all_tests, parallel_skip_tests))
logging.info("Found %s tests first 3 %s, parallel %s, other %s", len(all_tests), ' '.join(all_tests[:3]), len(filtered_parallel_tests), len(filtered_sequential_tests))
logging.info("Not found %s tests first 3 %s", len(not_found_tests), ' '.join(not_found_tests[:3]))
grouped_tests = self.group_test_by_file(filtered_sequential_tests)
i = 0
for par_group in chunks(filtered_parallel_tests, PARALLEL_GROUP_SIZE):
grouped_tests["parallel{}".format(i)] = par_group
i+=1
logging.info("Found %s tests groups", len(grouped_tests))
counters = {
@ -482,7 +515,7 @@ class ClickhouseIntegrationTestsRunner:
for group, tests in items_to_run:
logging.info("Running test group %s countaining %s tests", group, len(tests))
group_counters, group_test_times, log_paths = self.run_test_group(repo_path, group, tests, MAX_RETRY)
group_counters, group_test_times, log_paths = self.run_test_group(repo_path, group, tests, MAX_RETRY, NUM_WORKERS)
total_tests = 0
for counter, value in group_counters.items():
logging.info("Tests from group %s stats, %s count %s", group, counter, len(value))
@ -504,6 +537,10 @@ class ClickhouseIntegrationTestsRunner:
self._compress_logs("{}/tests/integration".format(repo_path), test_logs)
logging.info("Compression finished")
result_path_dockerd_logs = os.path.join(str(self.path()), "dockerd.log")
if os.path.exists(result_path_dockerd_logs):
shutil.copy(DOCKERD_LOGS_PATH, result_path_dockerd_logs)
if counters["FAILED"] or counters["ERROR"]:
logging.info("Overall status failure, because we have tests in FAILED or ERROR state")
result_state = "failure"

View File

@ -101,8 +101,8 @@ class CommandRequest:
self.stdout_file.seek(0)
self.stderr_file.seek(0)
stdout = self.stdout_file.read().decode()
stderr = self.stderr_file.read().decode()
stdout = self.stdout_file.read().decode('utf-8', errors='replace')
stderr = self.stderr_file.read().decode('utf-8', errors='replace')
if self.timer is not None and not self.process_finished_before_timeout and not self.ignore_error:
raise QueryTimeoutExceedException('Client timed out!')
@ -118,8 +118,8 @@ class CommandRequest:
self.stdout_file.seek(0)
self.stderr_file.seek(0)
stdout = self.stdout_file.read().decode()
stderr = self.stderr_file.read().decode()
stdout = self.stdout_file.read().decode('utf-8', errors='replace')
stderr = self.stderr_file.read().decode('utf-8', errors='replace')
if self.timer is not None and not self.process_finished_before_timeout and not self.ignore_error:
raise QueryTimeoutExceedException('Client timed out!')
@ -134,8 +134,8 @@ class CommandRequest:
self.stdout_file.seek(0)
self.stderr_file.seek(0)
stdout = self.stdout_file.read().decode()
stderr = self.stderr_file.read().decode()
stdout = self.stdout_file.read().decode('utf-8', errors='replace')
stderr = self.stderr_file.read().decode('utf-8', errors='replace')
if self.timer is not None and not self.process_finished_before_timeout and not self.ignore_error:
raise QueryTimeoutExceedException('Client timed out!')

File diff suppressed because it is too large Load Diff

View File

@ -7,12 +7,12 @@ class Layout(object):
'flat': '<flat/>',
'hashed': '<hashed/>',
'cache': '<cache><size_in_cells>128</size_in_cells></cache>',
'ssd_cache': '<ssd_cache><path>/etc/clickhouse/dictionaries/all</path></ssd_cache>',
'ssd_cache': '<ssd_cache><path>/etc/clickhouse-server/dictionaries/all</path></ssd_cache>',
'complex_key_hashed': '<complex_key_hashed/>',
'complex_key_hashed_one_key': '<complex_key_hashed/>',
'complex_key_hashed_two_keys': '<complex_key_hashed/>',
'complex_key_cache': '<complex_key_cache><size_in_cells>128</size_in_cells></complex_key_cache>',
'complex_key_ssd_cache': '<complex_key_ssd_cache><path>/etc/clickhouse/dictionaries/all</path></complex_key_ssd_cache>',
'complex_key_ssd_cache': '<complex_key_ssd_cache><path>/etc/clickhouse-server/dictionaries/all</path></complex_key_ssd_cache>',
'range_hashed': '<range_hashed/>',
'direct': '<direct/>',
'complex_key_direct': '<complex_key_direct/>'

View File

@ -9,6 +9,7 @@ import cassandra.cluster
import pymongo
import pymysql.cursors
import redis
import logging
from tzlocal import get_localzone
@ -59,6 +60,7 @@ class SourceMySQL(ExternalSource):
}
def create_mysql_conn(self):
logging.debug(f"pymysql connect {self.user}, {self.password}, {self.internal_hostname}, {self.internal_port}")
self.connection = pymysql.connect(
user=self.user,
password=self.password,
@ -98,8 +100,11 @@ class SourceMySQL(ExternalSource):
)
def prepare(self, structure, table_name, cluster):
if self.internal_hostname is None:
self.internal_hostname = cluster.mysql_ip
self.create_mysql_conn()
self.execute_mysql_query("create database if not exists test default character set 'utf8'")
self.execute_mysql_query("drop table if exists test.{}".format(table_name))
fields_strs = []
for field in structure.keys + structure.ordinary_fields + structure.range_fields:
fields_strs.append(field.name + ' ' + self.TYPE_MAPPING[field.field_type])
@ -457,6 +462,9 @@ class SourceCassandra(ExternalSource):
)
def prepare(self, structure, table_name, cluster):
if self.internal_hostname is None:
self.internal_hostname = cluster.cassandra_ip
self.client = cassandra.cluster.Cluster([self.internal_hostname], port=self.internal_port)
self.session = self.client.connect()
self.session.execute(

View File

@ -10,16 +10,6 @@ import socket
import tempfile
import logging
import os
g_dns_hook = None
def custom_getaddrinfo(*args):
# print("from custom_getaddrinfo g_dns_hook is None ", g_dns_hook is None)
ret = g_dns_hook.custom_getaddrinfo(*args)
# print("g_dns_hook.custom_getaddrinfo result", ret)
return ret
class mk_krb_conf(object):
def __init__(self, krb_conf, kdc_ip):
self.krb_conf = krb_conf
@ -37,32 +27,6 @@ class mk_krb_conf(object):
if self.amended_krb_conf is not None:
self.amended_krb_conf.close()
# tweak dns resolution to connect to localhost where api_host is in URL
class dns_hook(object):
def __init__(self, hdfs_api):
# print("dns_hook.init ", hdfs_api.kerberized, hdfs_api.host, hdfs_api.data_port, hdfs_api.proxy_port)
self.hdfs_api = hdfs_api
def __enter__(self):
global g_dns_hook
g_dns_hook = self
# print("g_dns_hook is None ", g_dns_hook is None)
self.original_getaddrinfo = socket.getaddrinfo
socket.getaddrinfo = custom_getaddrinfo
return self
def __exit__(self, type, value, traceback):
global g_dns_hook
g_dns_hook = None
socket.getaddrinfo = self.original_getaddrinfo
def custom_getaddrinfo(self, *args):
(hostname, port) = args[:2]
# print("top of custom_getaddrinfo", hostname, port)
if hostname == self.hdfs_api.host and (port == self.hdfs_api.data_port or port == self.hdfs_api.proxy_port):
# print("dns_hook substitute")
return [(socket.AF_INET, 1, 6, '', ("127.0.0.1", port))]
else:
return self.original_getaddrinfo(*args)
class HDFSApi(object):
def __init__(self, user, timeout=100, kerberized=False, principal=None,
keytab=None, krb_conf=None,
@ -84,12 +48,15 @@ class HDFSApi(object):
# logging.basicConfig(level=logging.DEBUG)
# logging.getLogger().setLevel(logging.DEBUG)
# requests_log = logging.getLogger("requests.packages.urllib3")
# requests_log.setLevel(logging.DEBUG)
# requests_log.setLevel(logging.INFO)
# requests_log.propagate = True
# kerb_log = logging.getLogger("requests_kerberos")
# kerb_log.setLevel(logging.DEBUG)
# kerb_log.propagate = True
if kerberized:
self._run_kinit()
self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override=self.host, principal=self.principal)
self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override="kerberizedhdfs1", principal=self.principal)
if self.kerberos_auth is None:
print("failed to obtain kerberos_auth")
else:
@ -100,51 +67,64 @@ class HDFSApi(object):
raise Exception("kerberos principal and keytab are required")
with mk_krb_conf(self.krb_conf, self.kdc_ip) as instantiated_krb_conf:
# print("instantiated_krb_conf ", instantiated_krb_conf)
logging.debug("instantiated_krb_conf {}".format(instantiated_krb_conf))
os.environ["KRB5_CONFIG"] = instantiated_krb_conf
cmd = "(kinit -R -t {keytab} -k {principal} || (sleep 5 && kinit -R -t {keytab} -k {principal})) ; klist".format(instantiated_krb_conf=instantiated_krb_conf, keytab=self.keytab, principal=self.principal)
# print(cmd)
start = time.time()
while time.time() - start < self.timeout:
try:
subprocess.call(cmd, shell=True)
print("KDC started, kinit successfully run")
res = subprocess.run(cmd, shell=True)
if res.returncode != 0:
# check_call(...) from subprocess does not print stderr, so we do it manually
logging.debug('Stderr:\n{}\n'.format(res.stderr.decode('utf-8')))
logging.debug('Stdout:\n{}\n'.format(res.stdout.decode('utf-8')))
logging.debug('Env:\n{}\n'.format(env))
raise Exception('Command {} return non-zero code {}: {}'.format(args, res.returncode, res.stderr.decode('utf-8')))
logging.debug("KDC started, kinit successfully run")
return
except Exception as ex:
print("Can't run kinit ... waiting {}".format(str(ex)))
logging.debug("Can't run kinit ... waiting {}".format(str(ex)))
time.sleep(1)
raise Exception("Kinit running failure")
def req_wrapper(self, func, expected_code, cnt=2, **kwargs):
with dns_hook(self):
for i in range(0, cnt):
response_data = func(**kwargs)
if response_data.status_code == expected_code:
return response_data
else:
print("unexpected response_data.status_code {}", response_data.status_code)
for i in range(0, cnt):
response_data = func(**kwargs)
if response_data.status_code == expected_code:
return response_data
else:
print("unexpected response_data.status_code {}", response_data.status_code)
response_data.raise_for_status()
def read_data(self, path, universal_newlines=True):
logging.debug("read_data protocol:{} host:{} port:{} path: {}".format(self.protocol, self.host, self.proxy_port, path))
response = self.req_wrapper(requests.get, 307, url="{protocol}://{host}:{port}/webhdfs/v1{path}?op=OPEN".format(protocol=self.protocol, host=self.host, port=self.proxy_port, path=path), headers={'host': 'localhost'}, allow_redirects=False, verify=False, auth=self.kerberos_auth)
# additional_params = '&'.join(response.headers['Location'].split('&')[1:2])
url = "{location}".format(location=response.headers['Location'])
# print("redirected to ", url)
response_data = self.req_wrapper(requests.get, 200, url=url,
headers={'host': 'localhost'},
verify=False, auth=self.kerberos_auth)
location = None
if self.kerberized:
location = response.headers['Location'].replace("kerberizedhdfs1:1006", "{}:{}".format(self.host, self.data_port))
else:
location = response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port))
logging.debug("redirected to {}".format(location))
response_data = self.req_wrapper(requests.get, 200, url=location, headers={'host': 'localhost'},
verify=False, auth=self.kerberos_auth)
if universal_newlines:
return response_data.text
else:
return response_data.content
def write_data(self, path, content):
logging.debug("write_data protocol:{} host:{} port:{} path: {} user:{}, principal:{}".format(
self.protocol, self.host, self.proxy_port, path, self.user, self.principal))
named_file = NamedTemporaryFile(mode='wb+')
fpath = named_file.name
if isinstance(content, str):
@ -152,36 +132,36 @@ class HDFSApi(object):
named_file.write(content)
named_file.flush()
if self.kerberized:
self._run_kinit()
self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override=self.host, principal=self.principal)
# print(self.kerberos_auth)
response = self.req_wrapper(requests.put, 307,
url="{protocol}://{host}:{port}/webhdfs/v1{path}?op=CREATE".format(
protocol=self.protocol, host=self.host,
port=self.proxy_port,
path=path, user=self.user),
allow_redirects=False,
headers={'host': 'localhost'},
params={'overwrite' : 'true'},
verify=False, auth=self.kerberos_auth
)
additional_params = '&'.join(
response.headers['Location'].split('&')[1:2] + ["user.name={}".format(self.user), "overwrite=true"])
url="{protocol}://{host}:{port}/webhdfs/v1{path}?op=CREATE".format(protocol=self.protocol, host='localhost',
port=self.proxy_port,
path=path, user=self.user),
allow_redirects=False,
headers={'host': 'localhost'},
params={'overwrite' : 'true'},
verify=False, auth=self.kerberos_auth
)
logging.debug("HDFS api response:{}".format(response.headers))
# additional_params = '&'.join(
# response.headers['Location'].split('&')[1:2] + ["user.name={}".format(self.user), "overwrite=true"])
if self.kerberized:
location = response.headers['Location'].replace("kerberizedhdfs1:1006", "{}:{}".format(self.host, self.data_port))
else:
location = response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port))
with open(fpath, mode="rb") as fh:
file_data = fh.read()
protocol = "http" # self.protocol
response = self.req_wrapper(requests.put, 201,
url="{location}".format(
location=response.headers['Location']),
data=file_data,
headers={'content-type':'text/plain', 'host': 'localhost'},
params={'file': path, 'user.name' : self.user},
allow_redirects=False, verify=False, auth=self.kerberos_auth
url="{location}".format(location=location),
data=file_data,
headers={'content-type':'text/plain', 'host': 'localhost'},
params={'file': path, 'user.name' : self.user},
allow_redirects=False, verify=False, auth=self.kerberos_auth
)
# print(response)
logging.debug(response)
def write_gzip_data(self, path, content):

View File

@ -1,7 +1,7 @@
import os
import subprocess
import time
import logging
import docker
@ -129,12 +129,12 @@ class _NetworkManager:
def add_iptables_rule(self, **kwargs):
cmd = ['iptables', '-I', 'DOCKER-USER', '1']
cmd.extend(self._iptables_cmd_suffix(**kwargs))
self._exec_run(cmd, privileged=True)
self._exec_run_with_retry(cmd, retry_count=3, privileged=True)
def delete_iptables_rule(self, **kwargs):
cmd = ['iptables', '-D', 'DOCKER-USER']
cmd.extend(self._iptables_cmd_suffix(**kwargs))
self._exec_run(cmd, privileged=True)
self._exec_run_with_retry(cmd, retry_count=3, privileged=True)
@staticmethod
def _iptables_cmd_suffix(
@ -159,12 +159,12 @@ class _NetworkManager:
def __init__(
self,
container_expire_timeout=50, container_exit_timeout=60):
container_expire_timeout=50, container_exit_timeout=60, docker_api_version=os.environ.get("DOCKER_API_VERSION")):
self.container_expire_timeout = container_expire_timeout
self.container_exit_timeout = container_exit_timeout
self._docker_client = docker.from_env(version=os.environ.get("DOCKER_API_VERSION"))
self._docker_client = docker.DockerClient(base_url='unix:///var/run/docker.sock', version=docker_api_version, timeout=600)
self._container = None
@ -207,6 +207,13 @@ class _NetworkManager:
return self._container
def _exec_run_with_retry(self, cmd, retry_count, **kwargs):
for i in range(retry_count):
try:
self._exec_run(cmd, **kwargs)
except subprocess.CalledProcessError as e:
logging.error(f"_exec_run failed for {cmd}, {e}")
def _exec_run(self, cmd, **kwargs):
container = self._ensure_container()
@ -219,3 +226,61 @@ class _NetworkManager:
raise subprocess.CalledProcessError(exit_code, cmd)
return output
# Approximately mesure network I/O speed for interface
class NetThroughput(object):
def __init__(self, node, interface="eth0"):
self.interface = interface
self.node = node
try:
check = subprocess.check_output(f'grep "^ *{self.interface}:" /proc/net/dev', shell=True)
if not check:
raise Exception(f"No such interface {self.interface} found in /proc/net/dev")
except:
raise Exception(f"No such interface {self.interface} found in /proc/net/dev")
self.current_in = self._get_in_bytes()
self.current_out = self._get_out_bytes()
self.measure_time = time.time()
def _get_in_bytes(self):
try:
result = self.node.exec_in_container(['bash', '-c', f'awk "/^ *{self.interface}:/"\' {{ if ($1 ~ /.*:[0-9][0-9]*/) {{ sub(/^.*:/, "") ; print $1 }} else {{ print $2 }} }}\' /proc/net/dev'])
except:
raise Exception(f"Cannot receive in bytes from /proc/net/dev for interface {self.interface}")
try:
return int(result)
except:
raise Exception(f"Got non-numeric in bytes '{result}' from /proc/net/dev for interface {self.interface}")
def _get_out_bytes(self):
try:
result = self.node.exec_in_container(['bash', '-c', f'awk "/^ *{self.interface}:/"\' {{ if ($1 ~ /.*:[0-9][0-9]*/) {{ print $9 }} else {{ print $10 }} }}\' /proc/net/dev'])
except:
raise Exception(f"Cannot receive out bytes from /proc/net/dev for interface {self.interface}")
try:
return int(result)
except:
raise Exception(f"Got non-numeric out bytes '{result}' from /proc/net/dev for interface {self.interface}")
def measure_speed(self, measure='bytes'):
new_in = self._get_in_bytes()
new_out = self._get_out_bytes()
current_time = time.time()
in_speed = (new_in - self.current_in) / (current_time - self.measure_time)
out_speed = (new_out - self.current_out) / (current_time - self.measure_time)
self.current_out = new_out
self.current_in = new_in
self.measure_time = current_time
if measure == 'bytes':
return in_speed, out_speed
elif measure == 'kilobytes':
return in_speed / 1024., out_speed / 1024.
elif measure == 'megabytes':
return in_speed / (1024 * 1024), out_speed / (1024 * 1024)
else:
raise Exception(f"Unknown measure {measure}")

View File

@ -0,0 +1,17 @@
<yandex>
<zookeeper>
<node index="1">
<host>zoo1</host>
<port>2281</port>
</node>
<node index="2">
<host>zoo2</host>
<port>2281</port>
</node>
<node index="3">
<host>zoo3</host>
<port>2281</port>
</node>
<session_timeout_ms>3000</session_timeout_ms>
</zookeeper>
</yandex>

View File

@ -0,0 +1,260 @@
[
"test_atomic_drop_table/test.py::test_atomic_delete_with_stopped_zookeeper",
"test_attach_without_fetching/test.py::test_attach_without_fetching",
"test_broken_part_during_merge/test.py::test_merge_and_part_corruption",
"test_cleanup_dir_after_bad_zk_conn/test.py::test_attach_without_zk",
"test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_bad_zk_conn",
"test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_replica_name",
"test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_zk_path",
"test_consistent_parts_after_clone_replica/test.py::test_inconsistent_parts_if_drop_while_replica_not_active",
"test_cross_replication/test.py::test",
"test_ddl_worker_non_leader/test.py::test_non_leader_replica",
"test_delayed_replica_failover/test.py::test",
"test_dictionaries_update_field/test.py::test_update_field[complex_key_hashed_update_field_dictionary-HASHED]",
"test_dictionaries_update_field/test.py::test_update_field[flat_update_field_dictionary-FLAT]",
"test_dictionaries_update_field/test.py::test_update_field[simple_key_hashed_update_field_dictionary-HASHED]",
"test_dictionary_allow_read_expired_keys/test_default_reading.py::test_default_reading",
"test_dictionary_allow_read_expired_keys/test_default_string.py::test_return_real_values",
"test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py::test_simple_dict_get_or_default",
"test_dictionary_allow_read_expired_keys/test_dict_get.py::test_simple_dict_get",
"test_disabled_mysql_server/test.py::test_disabled_mysql_server",
"test_distributed_ddl_on_cross_replication/test.py::test_alter_ddl",
"test_distributed_ddl_on_cross_replication/test.py::test_atomic_database",
"test_distributed_ddl_parallel/test.py::test_all_in_parallel",
"test_distributed_ddl_parallel/test.py::test_slow_dict_load_7",
"test_distributed_ddl_parallel/test.py::test_smoke",
"test_distributed_ddl_parallel/test.py::test_smoke_parallel",
"test_distributed_ddl_parallel/test.py::test_smoke_parallel_dict_reload",
"test_distributed_ddl_parallel/test.py::test_two_in_parallel_two_queued",
"test_distributed_ddl_password/test.py::test_alter",
"test_distributed_ddl_password/test.py::test_truncate",
"test_distributed_ddl/test.py::test_allowed_databases[configs]",
"test_distributed_ddl/test.py::test_allowed_databases[configs_secure]",
"test_distributed_ddl/test.py::test_create_as_select[configs]",
"test_distributed_ddl/test.py::test_create_as_select[configs_secure]",
"test_distributed_ddl/test.py::test_create_reserved[configs]",
"test_distributed_ddl/test.py::test_create_reserved[configs_secure]",
"test_distributed_ddl/test.py::test_create_view[configs]",
"test_distributed_ddl/test.py::test_create_view[configs_secure]",
"test_distributed_ddl/test.py::test_default_database[configs]",
"test_distributed_ddl/test.py::test_default_database[configs_secure]",
"test_distributed_ddl/test.py::test_detach_query[configs]",
"test_distributed_ddl/test.py::test_detach_query[configs_secure]",
"test_distributed_ddl/test.py::test_implicit_macros[configs]",
"test_distributed_ddl/test.py::test_implicit_macros[configs_secure]",
"test_distributed_ddl/test.py::test_kill_query[configs]",
"test_distributed_ddl/test.py::test_kill_query[configs_secure]",
"test_distributed_ddl/test.py::test_macro[configs]",
"test_distributed_ddl/test.py::test_macro[configs_secure]",
"test_distributed_ddl/test.py::test_on_connection_loss[configs]",
"test_distributed_ddl/test.py::test_on_connection_loss[configs_secure]",
"test_distributed_ddl/test.py::test_on_server_fail[configs]",
"test_distributed_ddl/test.py::test_on_server_fail[configs_secure]",
"test_distributed_ddl/test.py::test_on_session_expired[configs]",
"test_distributed_ddl/test.py::test_on_session_expired[configs_secure]",
"test_distributed_ddl/test.py::test_optimize_query[configs]",
"test_distributed_ddl/test.py::test_optimize_query[configs_secure]",
"test_distributed_ddl/test.py::test_rename[configs]",
"test_distributed_ddl/test.py::test_rename[configs_secure]",
"test_distributed_ddl/test.py::test_replicated_without_arguments[configs]",
"test_distributed_ddl/test.py::test_replicated_without_arguments[configs_secure]",
"test_distributed_ddl/test.py::test_simple_alters[configs]",
"test_distributed_ddl/test.py::test_simple_alters[configs_secure]",
"test_distributed_ddl/test.py::test_socket_timeout[configs]",
"test_distributed_ddl/test.py::test_socket_timeout[configs_secure]",
"test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs]",
"test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs_secure]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-distributed]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-remote]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-distributed]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-remote]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-distributed]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-remote]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-distributed]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-remote]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-distributed]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-remote]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-distributed]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-remote]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-distributed]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-remote]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-distributed]",
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-remote]",
"test_drop_replica/test.py::test_drop_replica",
"test_hedged_requests_parallel/test.py::test_combination1",
"test_hedged_requests_parallel/test.py::test_combination2",
"test_hedged_requests_parallel/test.py::test_query_with_no_data_to_sample",
"test_hedged_requests_parallel/test.py::test_send_data",
"test_hedged_requests_parallel/test.py::test_send_table_status_sleep",
"test_hedged_requests/test.py::test_combination1",
"test_hedged_requests/test.py::test_combination2",
"test_hedged_requests/test.py::test_combination3",
"test_hedged_requests/test.py::test_combination4",
"test_hedged_requests/test.py::test_long_query",
"test_hedged_requests/test.py::test_receive_timeout1",
"test_hedged_requests/test.py::test_receive_timeout2",
"test_hedged_requests/test.py::test_send_data",
"test_hedged_requests/test.py::test_send_data2",
"test_hedged_requests/test.py::test_send_table_status_sleep",
"test_hedged_requests/test.py::test_send_table_status_sleep2",
"test_hedged_requests/test.py::test_stuck_replica",
"test_https_replication/test.py::test_both_http",
"test_https_replication/test.py::test_both_https",
"test_https_replication/test.py::test_mixed_protocol",
"test_https_replication/test.py::test_replication_after_partition",
"test_insert_into_distributed_sync_async/test.py::test_async_inserts_into_local_shard",
"test_insert_into_distributed_sync_async/test.py::test_insertion_sync",
"test_insert_into_distributed_sync_async/test.py::test_insertion_sync_fails_with_timeout",
"test_insert_into_distributed_sync_async/test.py::test_insertion_sync_with_disabled_timeout",
"test_insert_into_distributed_sync_async/test.py::test_insertion_without_sync_ignores_timeout",
"test_insert_into_distributed/test.py::test_inserts_batching",
"test_insert_into_distributed/test.py::test_inserts_local",
"test_insert_into_distributed/test.py::test_inserts_low_cardinality",
"test_insert_into_distributed/test.py::test_inserts_single_replica_internal_replication",
"test_insert_into_distributed/test.py::test_inserts_single_replica_local_internal_replication",
"test_insert_into_distributed/test.py::test_inserts_single_replica_no_internal_replication",
"test_insert_into_distributed/test.py::test_prefer_localhost_replica",
"test_insert_into_distributed/test.py::test_reconnect",
"test_insert_into_distributed/test.py::test_table_function",
"test_insert_into_distributed_through_materialized_view/test.py::test_inserts_batching SKIPPED",
"test_insert_into_distributed_through_materialized_view/test.py::test_inserts_local",
"test_insert_into_distributed_through_materialized_view/test.py::test_reconnect",
"test_keeper_multinode_blocade_leader/test.py::test_blocade_leader",
"test_keeper_multinode_blocade_leader/test.py::test_blocade_leader_twice",
"test_keeper_multinode_simple/test.py::test_follower_restart",
"test_keeper_multinode_simple/test.py::test_read_write_multinode",
"test_keeper_multinode_simple/test.py::test_session_expiration",
"test_keeper_multinode_simple/test.py::test_simple_replicated_table",
"test_keeper_multinode_simple/test.py::test_watch_on_follower",
"test_limited_replicated_fetches/test.py::test_limited_fetches",
"test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node0]",
"test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node1]",
"test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node0]",
"test_materialize_mysql_database/test.py::test_mysql_settings[clickhouse_node1]",
"test_materialize_mysql_database/test.py::test_network_partition_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_network_partition_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_network_partition_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_network_partition_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_select_without_columns_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_select_without_columns_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_select_without_columns_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_select_without_columns_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node0]",
"test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node1]",
"test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0]",
"test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1]",
"test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node0]",
"test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node1]",
"test_parts_delete_zookeeper/test.py::test_merge_doesnt_work_without_zookeeper",
"test_polymorphic_parts/test.py::test_compact_parts_only",
"test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_compact-Compact]",
"test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_wide-Wide]",
"test_polymorphic_parts/test.py::test_in_memory",
"test_polymorphic_parts/test.py::test_in_memory_alters",
"test_polymorphic_parts/test.py::test_in_memory_deduplication",
"test_polymorphic_parts/test.py::test_in_memory_wal_rotate",
"test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node0-second_node0]",
"test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node1-second_node1]",
"test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions_2 SKIPPED",
"test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions SKIPPED",
"test_polymorphic_parts/test.py::test_polymorphic_parts_index",
"test_polymorphic_parts/test.py::test_polymorphic_parts_non_adaptive",
"test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_parallel",
"test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_quorum",
"test_random_inserts/test.py::test_insert_multithreaded",
"test_random_inserts/test.py::test_random_inserts",
"test_reload_clusters_config/test.py::test_add_cluster",
"test_reload_clusters_config/test.py::test_delete_cluster",
"test_reload_clusters_config/test.py::test_simple_reload",
"test_reload_clusters_config/test.py::test_update_one_cluster",
"test_replace_partition/test.py::test_drop_failover",
"test_replace_partition/test.py::test_normal_work",
"test_replace_partition/test.py::test_replace_after_replace_failover",
"test_replicated_database/test.py::test_alters_from_different_replicas",
"test_replicated_database/test.py::test_create_replicated_table",
"test_replicated_database/test.py::test_recover_staled_replica",
"test_replicated_database/test.py::test_simple_alter_table[MergeTree]",
"test_replicated_database/test.py::test_simple_alter_table[ReplicatedMergeTree]",
"test_replicated_database/test.py::test_startup_without_zk",
"test_replicated_fetches_timeouts/test.py::test_no_stall",
"test_storage_kafka/test.py::test_bad_reschedule",
"test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop",
"test_storage_kafka/test.py::test_exception_from_destructor",
"test_storage_kafka/test.py::test_kafka_commit_on_block_write",
"test_storage_kafka/test.py::test_kafka_consumer_hang",
"test_storage_kafka/test.py::test_kafka_consumer_hang2",
"test_storage_kafka/test.py::test_kafka_csv_with_delimiter",
"test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer",
"test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed",
"test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream",
"test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream_with_random_malformed_json",
"test_storage_kafka/test.py::test_kafka_flush_by_block_size",
"test_storage_kafka/test.py::test_kafka_flush_by_time",
"test_storage_kafka/test.py::test_kafka_flush_on_big_message",
"test_storage_kafka/test.py::test_kafka_formats",
"test_storage_kafka/test.py::test_kafka_formats_with_broken_message",
"test_storage_kafka/test.py::test_kafka_insert",
"test_storage_kafka/test.py::test_kafka_issue11308",
"test_storage_kafka/test.py::test_kafka_issue14202",
"test_storage_kafka/test.py::test_kafka_issue4116",
"test_storage_kafka/test.py::test_kafka_json_as_string",
"test_storage_kafka/test.py::test_kafka_json_without_delimiter",
"test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk",
"test_storage_kafka/test.py::test_kafka_many_materialized_views",
"test_storage_kafka/test.py::test_kafka_materialized_view",
"test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery",
"test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed",
"test_storage_kafka/test.py::test_kafka_produce_consume",
"test_storage_kafka/test.py::test_kafka_produce_key_timestamp",
"test_storage_kafka/test.py::test_kafka_protobuf",
"test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter",
"test_storage_kafka/test.py::test_kafka_rebalance",
"test_storage_kafka/test.py::test_kafka_select_empty",
"test_storage_kafka/test.py::test_kafka_settings_new_syntax",
"test_storage_kafka/test.py::test_kafka_settings_old_syntax",
"test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf",
"test_storage_kafka/test.py::test_kafka_tsv_with_delimiter",
"test_storage_kafka/test.py::test_kafka_unavailable",
"test_storage_kafka/test.py::test_kafka_virtual_columns",
"test_storage_kafka/test.py::test_kafka_virtual_columns2",
"test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view",
"test_storage_kafka/test.py::test_librdkafka_compression",
"test_storage_kafka/test.py::test_premature_flush_on_eof",
"test_storage_kerberized_kafka/test.py::test_kafka_json_as_string",
"test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc",
"test_system_clusters_actual_information/test.py::test",
"test_system_metrics/test.py::test_readonly_metrics",
"test_system_replicated_fetches/test.py::test_system_replicated_fetches"
]

View File

@ -0,0 +1,6 @@
# Parallel tests skip list is currently generated manually. All tests except those in parallel_skip.txt will run in parallel.
# Current list is generated with following commands
# 1. Generate all tests list as in CI run
./runner ' --setup-plan' | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' | sort -u > all_tests.txt
# 2. Filter known tests that are currently not run in parallel
cat all_tests.txt | grep '^test_replicated_database\|^test_disabled_mysql_server\|^test_distributed_ddl\|^test_distributed_ddl\|^test_quorum_inserts_parallel\|^test_ddl_worker_non_leader\|^test_consistent_parts_after_clone_replica\|^test_materialize_mysql_database\|^test_atomic_drop_table\|^test_distributed_respect_user_timeouts\|^test_storage_kafka\|^test_replace_partition\|^test_replicated_fetches_timeouts\|^test_system_clusters_actual_information\|^test_delayed_replica_failover\|^test_limited_replicated_fetches\|^test_hedged_requests\|^test_insert_into_distributed\|^test_insert_into_distributed_through_materialized_view\|^test_drop_replica\|^test_attach_without_fetching\|^test_system_replicated_fetches\|^test_cross_replication\|^test_dictionary_allow_read_expired_keys\|^test_dictionary_allow_read_expired_keys\|^test_dictionary_allow_read_expired_keys\|^test_insert_into_distributed_sync_async\|^test_hedged_requests_parallel\|^test_dictionaries_update_field\|^test_broken_part_during_merge\|^test_random_inserts\|^test_reload_clusters_config\|^test_parts_delete_zookeeper\|^test_polymorphic_parts\|^test_keeper_multinode_simple\|^test_https_replication\|^test_storage_kerberized_kafka\|^test_cleanup_dir_after_bad_zk_conn\|^test_system_metrics\|^test_keeper_multinode_blocade_leader' | awk '{$1=$1;print}' | jq -R -n '[inputs] | .' > parallel_skip.json

Some files were not shown because too many files have changed in this diff Show More