Merge branch 'master' into 100_percent_lld_11_for_clang_11

This commit is contained in:
alesapin 2020-09-28 13:41:01 +03:00
commit 29bc35ceff
288 changed files with 4431 additions and 2067 deletions

1
.gitmodules vendored
View File

@ -186,4 +186,3 @@
[submodule "contrib/cyrus-sasl"]
path = contrib/cyrus-sasl
url = https://github.com/cyrusimap/cyrus-sasl
branch = cyrus-sasl-2.1

View File

@ -173,7 +173,7 @@ endif ()
# Make sure the final executable has symbols exported
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic")
find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy")
find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-11" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy")
if (OBJCOPY_PATH)
message(STATUS "Using objcopy: ${OBJCOPY_PATH}.")
@ -313,7 +313,7 @@ if (COMPILER_CLANG)
endif ()
# Always prefer llvm tools when using clang. For instance, we cannot use GNU ar when llvm LTO is enabled
find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8")
find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8")
if (LLVM_AR_PATH)
message(STATUS "Using llvm-ar: ${LLVM_AR_PATH}.")
@ -322,7 +322,7 @@ if (COMPILER_CLANG)
message(WARNING "Cannot find llvm-ar. System ar will be used instead. It does not work with ThinLTO.")
endif ()
find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-10" "llvm-ranlib-9" "llvm-ranlib-8")
find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9" "llvm-ranlib-8")
if (LLVM_RANLIB_PATH)
message(STATUS "Using llvm-ranlib: ${LLVM_RANLIB_PATH}.")

View File

@ -358,7 +358,12 @@ private:
}
else
{
if (number * sizeof(base_type) < sizeof(T))
if constexpr (sizeof(T) <= sizeof(base_type))
{
if (!number)
return x;
}
else if (number * sizeof(base_type) < sizeof(T))
return x >> (number * base_bits); // & std::numeric_limits<base_type>::max()
return 0;
}
@ -366,26 +371,32 @@ private:
template <typename T>
constexpr static integer<Bits, Signed>
op_minus(const integer<Bits, Signed> & lhs, T rhs)
minus(const integer<Bits, Signed> & lhs, T rhs)
{
integer<Bits, Signed> res;
constexpr const unsigned rhs_items = (sizeof(T) > sizeof(base_type)) ? (sizeof(T) / sizeof(base_type)) : 1;
constexpr const unsigned op_items = (item_count < rhs_items) ? item_count : rhs_items;
bool is_underflow = false;
for (unsigned i = 0; i < item_count; ++i)
integer<Bits, Signed> res(lhs);
bool underflows[item_count] = {};
for (unsigned i = 0; i < op_items; ++i)
{
base_type lhs_item = lhs.items[little(i)];
base_type rhs_item = get_item(rhs, i);
base_type & res_item = res.items[little(i)];
if (is_underflow)
underflows[i] = res_item < rhs_item;
res_item -= rhs_item;
}
for (unsigned i = 1; i < item_count; ++i)
{
if (underflows[i-1])
{
is_underflow = (lhs_item == 0);
--lhs_item;
base_type & res_item = res.items[little(i)];
if (res_item == 0)
underflows[i] = true;
--res_item;
}
if (lhs_item < rhs_item)
is_underflow = true;
res.items[little(i)] = lhs_item - rhs_item;
}
return res;
@ -393,39 +404,44 @@ private:
template <typename T>
constexpr static integer<Bits, Signed>
op_plus(const integer<Bits, Signed> & lhs, T rhs)
plus(const integer<Bits, Signed> & lhs, T rhs)
{
integer<Bits, Signed> res;
constexpr const unsigned rhs_items = (sizeof(T) > sizeof(base_type)) ? (sizeof(T) / sizeof(base_type)) : 1;
constexpr const unsigned op_items = (item_count < rhs_items) ? item_count : rhs_items;
bool is_overflow = false;
for (unsigned i = 0; i < item_count; ++i)
integer<Bits, Signed> res(lhs);
bool overflows[item_count] = {};
for (unsigned i = 0; i < op_items; ++i)
{
base_type lhs_item = lhs.items[little(i)];
base_type rhs_item = get_item(rhs, i);
if (is_overflow)
{
++lhs_item;
is_overflow = (lhs_item == 0);
}
base_type & res_item = res.items[little(i)];
res_item = lhs_item + rhs_item;
if (res_item < rhs_item)
is_overflow = true;
res_item += rhs_item;
overflows[i] = res_item < rhs_item;
}
for (unsigned i = 1; i < item_count; ++i)
{
if (overflows[i-1])
{
base_type & res_item = res.items[little(i)];
++res_item;
if (res_item == 0)
overflows[i] = true;
}
}
return res;
}
template <typename T>
constexpr static auto op_multiply(const integer<Bits, Signed> & lhs, const T & rhs)
constexpr static auto multiply(const integer<Bits, Signed> & lhs, const T & rhs)
{
integer<Bits, Signed> res{};
#if 1
integer<Bits, Signed> lhs2 = op_plus(lhs, shift_left(lhs, 1));
integer<Bits, Signed> lhs3 = op_plus(lhs2, shift_left(lhs, 2));
integer<Bits, Signed> lhs2 = plus(lhs, shift_left(lhs, 1));
integer<Bits, Signed> lhs3 = plus(lhs2, shift_left(lhs, 2));
#endif
for (unsigned i = 0; i < item_count; ++i)
{
@ -437,7 +453,7 @@ private:
#if 1 /// optimization
if ((rhs_item & 0x7) == 0x7)
{
res = op_plus(res, shift_left(lhs3, pos));
res = plus(res, shift_left(lhs3, pos));
rhs_item >>= 3;
pos += 3;
continue;
@ -445,14 +461,14 @@ private:
if ((rhs_item & 0x3) == 0x3)
{
res = op_plus(res, shift_left(lhs2, pos));
res = plus(res, shift_left(lhs2, pos));
rhs_item >>= 2;
pos += 2;
continue;
}
#endif
if (rhs_item & 1)
res = op_plus(res, shift_left(lhs, pos));
res = plus(res, shift_left(lhs, pos));
rhs_item >>= 1;
++pos;
@ -475,7 +491,7 @@ public:
constexpr static integer<Bits, Signed>
operator_unary_minus(const integer<Bits, Signed> & lhs) noexcept(std::is_same_v<Signed, unsigned>)
{
return op_plus(operator_unary_tilda(lhs), 1);
return plus(operator_unary_tilda(lhs), 1);
}
template <typename T>
@ -484,9 +500,9 @@ public:
if constexpr (should_keep_size<T>())
{
if (is_negative(rhs))
return op_minus(lhs, -rhs);
return minus(lhs, -rhs);
else
return op_plus(lhs, rhs);
return plus(lhs, rhs);
}
else
{
@ -502,9 +518,9 @@ public:
if constexpr (should_keep_size<T>())
{
if (is_negative(rhs))
return op_plus(lhs, -rhs);
return plus(lhs, -rhs);
else
return op_minus(lhs, rhs);
return minus(lhs, rhs);
}
else
{
@ -523,12 +539,12 @@ public:
if constexpr (std::is_signed_v<Signed>)
{
res = op_multiply((is_negative(lhs) ? make_positive(lhs) : lhs),
res = multiply((is_negative(lhs) ? make_positive(lhs) : lhs),
(is_negative(rhs) ? make_positive(rhs) : rhs));
}
else
{
res = op_multiply(lhs, (is_negative(rhs) ? make_positive(rhs) : rhs));
res = multiply(lhs, (is_negative(rhs) ? make_positive(rhs) : rhs));
}
if (std::is_same_v<Signed, signed> && is_negative(lhs) != is_negative(rhs))
@ -775,20 +791,20 @@ public:
{
if (*c >= '0' && *c <= '9')
{
res = op_multiply(res, 16U);
res = op_plus(res, *c - '0');
res = multiply(res, 16U);
res = plus(res, *c - '0');
++c;
}
else if (*c >= 'a' && *c <= 'f')
{
res = op_multiply(res, 16U);
res = op_plus(res, *c - 'a' + 10U);
res = multiply(res, 16U);
res = plus(res, *c - 'a' + 10U);
++c;
}
else if (*c >= 'A' && *c <= 'F')
{ // tolower must be used, but it is not constexpr
res = op_multiply(res, 16U);
res = op_plus(res, *c - 'A' + 10U);
res = multiply(res, 16U);
res = plus(res, *c - 'A' + 10U);
++c;
}
else
@ -802,8 +818,8 @@ public:
if (*c < '0' || *c > '9')
throwError("invalid char from");
res = op_multiply(res, 10U);
res = op_plus(res, *c - '0');
res = multiply(res, 10U);
res = plus(res, *c - '0');
++c;
}
}

View File

@ -14,6 +14,8 @@ TRIES=3
AMD64_BIN_URL="https://clickhouse-builds.s3.yandex.net/0/e29c4c3cc47ab2a6c4516486c1b77d57e7d42643/clickhouse_build_check/gcc-10_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse"
AARCH64_BIN_URL="https://clickhouse-builds.s3.yandex.net/0/e29c4c3cc47ab2a6c4516486c1b77d57e7d42643/clickhouse_special_build_check/clang-10-aarch64_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse"
# Note: on older Ubuntu versions, 'axel' does not support IPv6. If you are using IPv6-only servers on very old Ubuntu, just don't install 'axel'.
FASTER_DOWNLOAD=wget
if command -v axel >/dev/null; then
FASTER_DOWNLOAD=axel
@ -36,14 +38,6 @@ if [[ ! -f clickhouse ]]; then
$FASTER_DOWNLOAD "$AMD64_BIN_URL"
elif [[ $CPU == aarch64 ]]; then
$FASTER_DOWNLOAD "$AARCH64_BIN_URL"
# Download configs. ARM version has no embedded configs.
wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/config.xml
wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/users.xml
mkdir config.d
wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/config.d/path.xml -O config.d/path.xml
wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/config.d/access_control.xml -O config.d/access_control.xml
wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/config.d/log_to_console.xml -O config.d/log_to_console.xml
else
echo "Unsupported CPU type: $CPU"
exit 1
@ -60,10 +54,12 @@ if [[ ! -d data ]]; then
if [[ ! -f $DATASET ]]; then
$FASTER_DOWNLOAD "https://clickhouse-datasets.s3.yandex.net/hits/partitions/$DATASET"
fi
tar $TAR_PARAMS --strip-components=1 --directory=. -x -v -f $DATASET
fi
uptime
echo "Starting clickhouse-server"
./clickhouse server > server.log 2>&1 &
@ -105,9 +101,12 @@ echo
echo "Benchmark complete. System info:"
echo
echo '----Version and build id--------'
./clickhouse local --query "SELECT version(), buildId()"
echo '----Version, build id-----------'
./clickhouse local --query "SELECT format('Version: {}, build id: {}', version(), buildId())"
./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw
./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))"
echo '----CPU-------------------------'
cat /proc/cpuinfo | grep -i -F 'model name' | uniq
lscpu
echo '----Block Devices---------------'
lsblk

View File

@ -14,10 +14,10 @@ if (NOT ENABLE_RDKAFKA)
return()
endif()
if (NOT ARCH_ARM)
if (NOT ARCH_ARM AND USE_LIBGSASL)
option (USE_INTERNAL_RDKAFKA_LIBRARY "Set to FALSE to use system librdkafka instead of the bundled" ${NOT_UNBUNDLED})
elseif(USE_INTERNAL_RDKAFKA_LIBRARY)
message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal librdkafka with ARCH_ARM=${ARCH_ARM}")
message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal librdkafka with ARCH_ARM=${ARCH_ARM} AND USE_LIBGSASL=${USE_LIBGSASL}")
endif ()
if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cppkafka/CMakeLists.txt")

2
contrib/cyrus-sasl vendored

@ -1 +1 @@
Subproject commit 9995bf9d8e14f58934d9313ac64f13780d6dd3c9
Subproject commit 6054630889fd1cd8d0659573d69badcee1e23a00

2
contrib/protobuf vendored

@ -1 +1 @@
Subproject commit d6a10dd3db55d8f7f9e464db9151874cde1f79ec
Subproject commit 445d1ae73a450b1e94622e7040989aa2048402e3

View File

@ -11,3 +11,7 @@ else ()
endif ()
add_subdirectory("${protobuf_SOURCE_DIR}/cmake" "${protobuf_BINARY_DIR}")
# We don't want to stop compilation on warnings in protobuf's headers.
# The following line overrides the value assigned by the command target_include_directories() in libprotobuf.cmake
set_property(TARGET libprotobuf PROPERTY INTERFACE_SYSTEM_INCLUDE_DIRECTORIES ${protobuf_SOURCE_DIR}/src)

4
debian/rules vendored
View File

@ -36,8 +36,8 @@ endif
CMAKE_FLAGS += -DENABLE_UTILS=0
DEB_CC ?= $(shell which gcc-9 gcc-8 gcc | head -n1)
DEB_CXX ?= $(shell which g++-9 g++-8 g++ | head -n1)
DEB_CC ?= $(shell which gcc-10 gcc-9 gcc | head -n1)
DEB_CXX ?= $(shell which g++-10 g++-9 g++ | head -n1)
ifdef DEB_CXX
DEB_BUILD_GNU_TYPE := $(shell dpkg-architecture -qDEB_BUILD_GNU_TYPE)

View File

@ -133,10 +133,6 @@
"name": "yandex/clickhouse-postgresql-java-client",
"dependent": []
},
"docker/test/integration/kerberos_kdc": {
"name": "yandex/clickhouse-kerberos-kdc",
"dependent": []
},
"docker/test/base": {
"name": "yandex/clickhouse-test-base",
"dependent": [

View File

@ -89,7 +89,8 @@ EOT
fi
if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then
$gosu /usr/bin/clickhouse-server --config-file=$CLICKHOUSE_CONFIG &
# Listen only on localhost until the initialization is done
$gosu /usr/bin/clickhouse-server --config-file=$CLICKHOUSE_CONFIG -- --listen_host=127.0.0.1 &
pid="$!"
# check if clickhouse is ready to accept connections

View File

@ -97,7 +97,7 @@ ccache --zero-stats ||:
mkdir build
cd build
cmake .. -DCMAKE_INSTALL_PREFIX=/usr -DCMAKE_CXX_COMPILER=clang++-10 -DCMAKE_C_COMPILER=clang-10 "${CMAKE_LIBS_CONFIG[@]}" "${FASTTEST_CMAKE_FLAGS[@]}" | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/cmake_log.txt
ninja clickhouse-bundle | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/build_log.txt
time ninja clickhouse-bundle | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/build_log.txt
ninja install | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/install_log.txt
@ -111,35 +111,10 @@ ln -s /test_output /var/log/clickhouse-server
cp "$CLICKHOUSE_DIR/programs/server/config.xml" /etc/clickhouse-server/
cp "$CLICKHOUSE_DIR/programs/server/users.xml" /etc/clickhouse-server/
mkdir -p /etc/clickhouse-server/dict_examples
ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/
ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/
ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/
ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/custom_settings_prefixes.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/
#ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/clusters.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/
ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml
# Keep original query_masking_rules.xml
ln -s --backup=simple --suffix=_original.xml /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/
# install tests config
$CLICKHOUSE_DIR/tests/config/install.sh
# doesn't support SSL
rm -f /etc/clickhouse-server/config.d/secure_ports.xml
# Kill the server in case we are running locally and not in docker
kill_clickhouse
@ -216,7 +191,7 @@ TESTS_TO_SKIP=(
01460_DistributedFilesToInsert
)
clickhouse-test -j 8 --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/test_log.txt
time clickhouse-test -j 8 --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/test_log.txt
# substr is to remove semicolon after test name
@ -234,7 +209,7 @@ then
kill_clickhouse
# Clean the data so that there is no interference from the previous test run.
rm -rvf /var/lib/clickhouse ||:
rm -rf /var/lib/clickhouse ||:
mkdir /var/lib/clickhouse
clickhouse-server --config /etc/clickhouse-server/config.xml --daemon

View File

@ -48,7 +48,7 @@ function configure
cp -av "$repo_dir"/programs/server/config* db
cp -av "$repo_dir"/programs/server/user* db
# TODO figure out which ones are needed
cp -av "$repo_dir"/tests/config/listen.xml db/config.d
cp -av "$repo_dir"/tests/config/config.d/listen.xml db/config.d
cp -av "$script_dir"/query-fuzzer-tweaks-users.xml db/users.d
}

View File

@ -16,8 +16,7 @@ RUN apt-get update \
odbc-postgresql \
sqlite3 \
curl \
tar \
krb5-user
tar
RUN rm -rf \
/var/lib/apt/lists/* \
/var/cache/debconf \

View File

@ -1,15 +0,0 @@
# docker build -t yandex/clickhouse-kerberos-kdc .
FROM centos:6.6
# old OS to make is faster and smaller
RUN yum install -y krb5-server krb5-libs krb5-auth-dialog krb5-workstation
EXPOSE 88 749
RUN touch /config.sh
# should be overwritten e.g. via docker_compose volumes
# volumes: /some_path/my_kerberos_config.sh:/config.sh:ro
ENTRYPOINT ["/bin/bash", "/config.sh"]

View File

@ -1,59 +0,0 @@
version: '2.3'
services:
kafka_kerberized_zookeeper:
image: confluentinc/cp-zookeeper:5.2.0
# restart: always
hostname: kafka_kerberized_zookeeper
environment:
ZOOKEEPER_SERVER_ID: 1
ZOOKEEPER_CLIENT_PORT: 2181
ZOOKEEPER_SERVERS: "kafka_kerberized_zookeeper:2888:3888"
KAFKA_OPTS: "-Djava.security.auth.login.config=/etc/kafka/secrets/zookeeper_jaas.conf -Djava.security.krb5.conf=/etc/kafka/secrets/krb.conf -Dzookeeper.authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider -Dsun.security.krb5.debug=true"
volumes:
- ${KERBERIZED_KAFKA_DIR}/secrets:/etc/kafka/secrets
- /dev/urandom:/dev/random
depends_on:
- kafka_kerberos
security_opt:
- label:disable
kerberized_kafka1:
image: confluentinc/cp-kafka:5.2.0
# restart: always
hostname: kerberized_kafka1
ports:
- "9092:9092"
- "9093:9093"
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: INSIDE://kerberized_kafka1:9092,OUTSIDE://kerberized_kafka1:19092
# KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kerberized_kafka1:19092
KAFKA_SASL_MECHANISM_INTER_BROKER_PROTOCOL: GSSAPI
KAFKA_SASL_ENABLED_MECHANISMS: GSSAPI
KAFKA_SASL_KERBEROS_SERVICE_NAME: kafka
KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: OUTSIDE:SASL_PLAINTEXT,UNSECURED_OUTSIDE:PLAINTEXT,UNSECURED_INSIDE:PLAINTEXT,
KAFKA_INTER_BROKER_LISTENER_NAME: OUTSIDE
KAFKA_BROKER_ID: 1
KAFKA_ZOOKEEPER_CONNECT: "kafka_kerberized_zookeeper:2181"
KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO"
KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1
KAFKA_OPTS: "-Djava.security.auth.login.config=/etc/kafka/secrets/broker_jaas.conf -Djava.security.krb5.conf=/etc/kafka/secrets/krb.conf -Dsun.security.krb5.debug=true"
volumes:
- ${KERBERIZED_KAFKA_DIR}/secrets:/etc/kafka/secrets
- /dev/urandom:/dev/random
depends_on:
- kafka_kerberized_zookeeper
- kafka_kerberos
security_opt:
- label:disable
kafka_kerberos:
image: yandex/clickhouse-kerberos-kdc:${DOCKER_KERBEROS_KDC_TAG}
hostname: kafka_kerberos
volumes:
- ${KERBERIZED_KAFKA_DIR}/secrets:/tmp/keytab
- ${KERBERIZED_KAFKA_DIR}/../../kerberos_image_config.sh:/config.sh
- /dev/urandom:/dev/random
ports: [88, 749]

View File

@ -27,7 +27,6 @@ export DOCKER_MYSQL_JAVA_CLIENT_TAG=${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest}
export DOCKER_MYSQL_JS_CLIENT_TAG=${DOCKER_MYSQL_JS_CLIENT_TAG:=latest}
export DOCKER_MYSQL_PHP_CLIENT_TAG=${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest}
export DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest}
export DOCKER_KERBEROS_KDC_TAG=${DOCKER_KERBEROS_KDC_TAG:=latest}
cd /ClickHouse/tests/integration
exec "$@"

View File

@ -24,10 +24,11 @@ def tsv_escape(s):
parser = argparse.ArgumentParser(description='Run performance test.')
# Explicitly decode files as UTF-8 because sometimes we have Russian characters in queries, and LANG=C is set.
parser.add_argument('file', metavar='FILE', type=argparse.FileType('r', encoding='utf-8'), nargs=1, help='test description file')
parser.add_argument('--host', nargs='*', default=['localhost'], help="Server hostname(s). Corresponds to '--port' options.")
parser.add_argument('--port', nargs='*', default=[9000], help="Server port(s). Corresponds to '--host' options.")
parser.add_argument('--host', nargs='*', default=['localhost'], help="Space-separated list of server hostname(s). Corresponds to '--port' options.")
parser.add_argument('--port', nargs='*', default=[9000], help="Space-separated list of server port(s). Corresponds to '--host' options.")
parser.add_argument('--runs', type=int, default=1, help='Number of query runs per server.')
parser.add_argument('--max-queries', type=int, default=None, help='Test no more than this number of queries, chosen at random.')
parser.add_argument('--queries-to-run', nargs='*', type=int, default=None, help='Space-separated list of indexes of queries to test.')
parser.add_argument('--long', action='store_true', help='Do not skip the tests tagged as long.')
parser.add_argument('--print-queries', action='store_true', help='Print test queries and exit.')
parser.add_argument('--print-settings', action='store_true', help='Print test settings and exit.')
@ -188,10 +189,20 @@ for t in threads:
for t in threads:
t.join()
# Run the queries in randomized order, but preserve their indexes as specified
# in the test XML. To avoid using too much time, limit the number of queries
# we run per test.
queries_to_run = random.sample(range(0, len(test_queries)), min(len(test_queries), args.max_queries or len(test_queries)))
queries_to_run = range(0, len(test_queries))
if args.max_queries:
# If specified, test a limited number of queries chosen at random.
queries_to_run = random.sample(range(0, len(test_queries)), min(len(test_queries), args.max_queries))
if args.queries_to_run:
# Run the specified queries, with some sanity check.
for i in args.queries_to_run:
if i < 0 or i >= len(test_queries):
print(f'There is no query no. "{i}" in this test, only [{0}-{len(test_queries) - 1}] are present')
exit(1)
queries_to_run = args.queries_to_run
# Run test queries.
for query_index in queries_to_run:

View File

@ -8,26 +8,8 @@ dpkg -i package_folder/clickhouse-server_*.deb
dpkg -i package_folder/clickhouse-client_*.deb
dpkg -i package_folder/clickhouse-test_*.deb
mkdir -p /etc/clickhouse-server/dict_examples
ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/
ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/
ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/
ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/
if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then
ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/
fi
# install test configs
/usr/share/clickhouse-test/config/install.sh
function start()
{

View File

@ -48,28 +48,8 @@ mkdir -p /var/lib/clickhouse
mkdir -p /var/log/clickhouse-server
chmod 777 -R /var/log/clickhouse-server/
# Temorary way to keep CI green while moving dictionaries to separate directory
mkdir -p /etc/clickhouse-server/dict_examples
chmod 777 -R /etc/clickhouse-server/dict_examples
ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/; \
ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/; \
ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/;
ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/
# Retain any pre-existing config and allow ClickHouse to load those if required
ln -s --backup=simple --suffix=_original.xml \
/usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/
# install test configs
/usr/share/clickhouse-test/config/install.sh
function start()
{

View File

@ -21,9 +21,7 @@ RUN apt-get update -y \
telnet \
tree \
unixodbc \
wget \
zookeeper \
zookeeperd
wget
RUN mkdir -p /tmp/clickhouse-odbc-tmp \
&& wget -nv -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \

View File

@ -8,48 +8,9 @@ dpkg -i package_folder/clickhouse-server_*.deb
dpkg -i package_folder/clickhouse-client_*.deb
dpkg -i package_folder/clickhouse-test_*.deb
mkdir -p /etc/clickhouse-server/dict_examples
ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/
ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/
ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/
ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/custom_settings_prefixes.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/clusters.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/
# install test configs
/usr/share/clickhouse-test/config/install.sh
# Retain any pre-existing config and allow ClickHouse to load it if required
ln -s --backup=simple --suffix=_original.xml \
/usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/
if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then
ln -s /usr/share/clickhouse-test/config/polymorphic_parts.xml /etc/clickhouse-server/config.d/
fi
if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then
ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/
fi
ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml
service zookeeper start
sleep 5
service clickhouse-server start && sleep 5
if cat /usr/bin/clickhouse-test | grep -q -- "--use-skip-list"; then

View File

@ -66,9 +66,7 @@ RUN apt-get --allow-unauthenticated update -y \
unixodbc \
unixodbc-dev \
wget \
zlib1g-dev \
zookeeper \
zookeeperd
zlib1g-dev
RUN mkdir -p /tmp/clickhouse-odbc-tmp \
&& wget -nv -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \

View File

@ -8,48 +8,9 @@ dpkg -i package_folder/clickhouse-server_*.deb
dpkg -i package_folder/clickhouse-client_*.deb
dpkg -i package_folder/clickhouse-test_*.deb
mkdir -p /etc/clickhouse-server/dict_examples
ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/
ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/
ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/
ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/custom_settings_prefixes.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/clusters.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/
# install test configs
/usr/share/clickhouse-test/config/install.sh
# Retain any pre-existing config and allow ClickHouse to load it if required
ln -s --backup=simple --suffix=_original.xml \
/usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/
if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then
ln -s /usr/share/clickhouse-test/config/polymorphic_parts.xml /etc/clickhouse-server/config.d/
fi
if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then
ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/
fi
ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml
service zookeeper start
sleep 5
service clickhouse-server start && sleep 5
if cat /usr/bin/clickhouse-test | grep -q -- "--use-skip-list"; then

View File

@ -11,8 +11,6 @@ RUN apt-get update -y \
tzdata \
fakeroot \
debhelper \
zookeeper \
zookeeperd \
expect \
python \
python-lxml \

View File

@ -39,41 +39,8 @@ mkdir -p /var/log/clickhouse-server
chmod 777 -R /var/lib/clickhouse
chmod 777 -R /var/log/clickhouse-server/
# Temorary way to keep CI green while moving dictionaries to separate directory
mkdir -p /etc/clickhouse-server/dict_examples
chmod 777 -R /etc/clickhouse-server/dict_examples
ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/; \
ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/; \
ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/;
ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/clusters.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/
ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/
ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml
# Retain any pre-existing config and allow ClickHouse to load it if required
ln -s --backup=simple --suffix=_original.xml \
/usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/
service zookeeper start
sleep 5
# install test configs
/usr/share/clickhouse-test/config/install.sh
start_clickhouse

View File

@ -39,9 +39,8 @@ function start()
done
}
ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/
# install test configs
/usr/share/clickhouse-test/config/install.sh
echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment

View File

@ -29,7 +29,7 @@ def get_options(i):
if 0 < i:
options += " --order=random"
if i % 2 == 1:
options += " --atomic-db-engine"
options += " --db-engine=Ordinary"
return options

View File

@ -35,7 +35,7 @@ RUN apt-get update \
ENV TZ=Europe/Moscow
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
RUN pip3 install urllib3 testflows==1.6.42 docker-compose docker dicttoxml kazoo tzlocal
RUN pip3 install urllib3 testflows==1.6.48 docker-compose docker dicttoxml kazoo tzlocal
ENV DOCKER_CHANNEL stable
ENV DOCKER_VERSION 17.09.1-ce

View File

@ -165,22 +165,6 @@ Similar to GraphiteMergeTree, the Kafka engine supports extended configuration u
For a list of possible configuration options, see the [librdkafka configuration reference](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md). Use the underscore (`_`) instead of a dot in the ClickHouse configuration. For example, `check.crcs=true` will be `<check_crcs>true</check_crcs>`.
### Kerberos support {#kafka-kerberos-support}
To deal with Kerberos-aware Kafka, add `security_protocol` child element with `sasl_plaintext` value. It is enough if Kerberos ticket-granting ticket is obtained and cached by OS facilities.
ClickHouse is able to maintain Kerberos credentials using a keytab file. Consider `sasl_kerberos_service_name`, `sasl_kerberos_keytab`, `sasl_kerberos_principal` and `sasl.kerberos.kinit.cmd` child elements.
Example:
``` xml
<!-- Kerberos-aware Kafka -->
<kafka>
<security_protocol>SASL_PLAINTEXT</security_protocol>
<sasl_kerberos_keytab>/home/kafkauser/kafkauser.keytab</sasl_kerberos_keytab>
<sasl_kerberos_principal>kafkauser/kafkahost@EXAMPLE.COM</sasl_kerberos_principal>
</kafka>
```
## Virtual Columns {#virtual-columns}
- `_topic` — Kafka topic.

View File

@ -38,7 +38,7 @@ toc_title: Adopters
| <a href="https://db.com" class="favicon">Deutsche Bank</a> | Finance | BI Analytics | — | — | [Slides in English, October 2019](https://bigdatadays.ru/wp-content/uploads/2019/10/D2-H3-3_Yakunin-Goihburg.pdf) |
| <a href="https://www.diva-e.com" class="favicon">Diva-e</a> | Digital consulting | Main Product | — | — | [Slides in English, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup29/ClickHouse-MeetUp-Unusual-Applications-sd-2019-09-17.pdf) |
| <a href="https://www.ecwid.com/" class="favicon">Ecwid</a> | E-commerce SaaS | Metrics, Logging | — | — | [Slides in Russian, April 2019](https://nastachku.ru/var/files/1/presentation/backend/2_Backend_6.pdf) |
| <a href="https://www.ebay.com/" class="favicon">eBay</a> | E-commerce | TBA | — | — | [Webinar, Sep 2020](https://altinity.com/webinarspage/2020/09/08/migrating-from-druid-to-next-gen-olap-on-clickhouse-ebays-experience) |
| <a href="https://www.ebay.com/" class="favicon">eBay</a> | E-commerce | Logs, Metrics and Events | — | — | [Official website, Sep 2020](https://tech.ebayinc.com/engineering/ou-online-analytical-processing/) |
| <a href="https://www.exness.com" class="favicon">Exness</a> | Trading | Metrics, Logging | — | — | [Talk in Russian, May 2019](https://youtu.be/_rpU-TvSfZ8?t=3215) |
| <a href="https://fastnetmon.com/" class="favicon">FastNetMon</a> | DDoS Protection | Main Product | | — | [Official website](https://fastnetmon.com/docs-fnm-advanced/fastnetmon-advanced-traffic-persistency/) |
| <a href="https://www.flipkart.com/" class="favicon">Flipkart</a> | e-Commerce | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=239) |

View File

@ -0,0 +1,10 @@
---
toc_priority: 100
---
# Information support {#information-support}
- Email address: <a class="feedback-email"></a>
- Phone: <a href="tel:+74957806510">+7-495-780-6510</a>
[Original article](https://clickhouse.tech/docs/en/introduction/info/) <!--hide-->

View File

@ -7,6 +7,6 @@ toc_priority: 100
Информационная поддержка ClickHouse осуществляется на всей территории Российской Федерации без ограничений посредством использования телефонной связи и средств электронной почты на русском языке в круглосуточном режиме:
- Адрес электронной почты: <a class="feedback-email"></a>
- Телефон: <a href="tel:88002509639">8-800-250-96-39</a> (звонки бесплатны из всех регионов России)
- Телефон: <a href="tel:+74957806510">+7-495-780-6510</a>
[Оригинальная статья](https://clickhouse.tech/docs/ru/introduction/info/) <!--hide-->

View File

@ -49,13 +49,14 @@ def translate_impl(text, target_language=None):
def translate(text, target_language=None):
result = []
for part in re.split(curly_braces_re, text):
if part.startswith('{') and part.endswith('}'):
result.append(part)
else:
result.append(translate_impl(part, target_language=target_language))
return ''.join(result)
return "".join(
[
part
if part.startswith("{") and part.endswith("}")
else translate_impl(part, target_language=target_language)
for part in re.split(curly_braces_re, text)
]
)
def translate_toc(root, lang):

View File

@ -3,7 +3,7 @@ machine_translated: true
machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3
---
# 在运营商 {#select-in-operators}
# IN 操作符 {#select-in-operators}
`IN`, `NOT IN`, `GLOBAL IN`,和 `GLOBAL NOT IN` 运算符是单独复盖的,因为它们的功能相当丰富。

View File

@ -80,7 +80,7 @@ Suggest::Suggest()
"WITH", "TOTALS", "HAVING", "ORDER", "COLLATE", "LIMIT", "UNION", "AND", "OR", "ASC",
"IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE",
"PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE",
"IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "FOR", "RANDOMIZED",
"IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "RANDOMIZED",
"INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE"};
}

View File

@ -1477,7 +1477,9 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
{
auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query,
task_table.table_push, task_table.engine_push_ast);
create_query_push_ast->as<ASTCreateQuery &>().if_not_exists = true;
auto & create = create_query_push_ast->as<ASTCreateQuery &>();
create.if_not_exists = true;
InterpreterCreateQuery::prepareOnClusterQuery(create, context, task_table.cluster_push_name);
String query = queryToString(create_query_push_ast);
LOG_DEBUG(log, "Create destination tables. Query: {}", query);

View File

@ -215,31 +215,20 @@ Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast)
return primary_key_columns;
}
String extractReplicatedTableZookeeperPath(const ASTPtr & storage_ast)
bool isReplicatedTableEngine(const ASTPtr & storage_ast)
{
String storage_str = queryToString(storage_ast);
const auto & storage = storage_ast->as<ASTStorage &>();
const auto & engine = storage.engine->as<ASTFunction &>();
if (!endsWith(engine.name, "MergeTree"))
{
String storage_str = queryToString(storage_ast);
throw Exception(
"Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported",
ErrorCodes::BAD_ARGUMENTS);
}
if (!startsWith(engine.name, "Replicated"))
{
return "";
}
auto replicated_table_arguments = engine.arguments->children;
auto zk_table_path_ast = replicated_table_arguments[0]->as<ASTLiteral &>();
auto zk_table_path_string = zk_table_path_ast.value.safeGet<String>();
return zk_table_path_string;
return startsWith(engine.name, "Replicated");
}
ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random)

View File

@ -200,7 +200,7 @@ ASTPtr extractOrderBy(const ASTPtr & storage_ast);
Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast);
String extractReplicatedTableZookeeperPath(const ASTPtr & storage_ast);
bool isReplicatedTableEngine(const ASTPtr & storage_ast);
ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random);

View File

@ -48,7 +48,7 @@ struct TaskTable
String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const;
bool isReplicatedTable() const { return engine_push_zk_path != ""; }
bool isReplicatedTable() const { return is_replicated_table; }
/// Partitions will be split into number-of-splits pieces.
/// Each piece will be copied independently. (10 by default)
@ -78,6 +78,7 @@ struct TaskTable
/// First argument of Replicated...MergeTree()
String engine_push_zk_path;
bool is_replicated_table;
ASTPtr rewriteReplicatedCreateQueryToPlain();
@ -269,7 +270,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf
engine_push_ast = parseQuery(parser_storage, engine_push_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
engine_push_partition_key_ast = extractPartitionKey(engine_push_ast);
primary_key_comma_separated = Nested::createCommaSeparatedStringFrom(extractPrimaryKeyColumnNames(engine_push_ast));
engine_push_zk_path = extractReplicatedTableZookeeperPath(engine_push_ast);
is_replicated_table = isReplicatedTableEngine(engine_push_ast);
}
sharding_key_str = config.getString(table_prefix + "sharding_key");
@ -372,15 +373,18 @@ inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain()
auto & new_storage_ast = prev_engine_push_ast->as<ASTStorage &>();
auto & new_engine_ast = new_storage_ast.engine->as<ASTFunction &>();
auto & replicated_table_arguments = new_engine_ast.arguments->children;
/// Delete first two arguments of Replicated...MergeTree() table.
replicated_table_arguments.erase(replicated_table_arguments.begin());
replicated_table_arguments.erase(replicated_table_arguments.begin());
/// Remove replicated from name
/// Remove "Replicated" from name
new_engine_ast.name = new_engine_ast.name.substr(10);
if (new_engine_ast.arguments)
{
auto & replicated_table_arguments = new_engine_ast.arguments->children;
/// Delete first two arguments of Replicated...MergeTree() table.
replicated_table_arguments.erase(replicated_table_arguments.begin());
replicated_table_arguments.erase(replicated_table_arguments.begin());
}
return new_storage_ast.clone();
}

View File

@ -5,6 +5,9 @@ set (CLICKHOUSE_FORMAT_LINK
boost::program_options
clickhouse_common_io
clickhouse_parsers
clickhouse_functions
clickhouse_aggregate_functions
clickhouse_table_functions
dbms
)

View File

@ -1,13 +1,29 @@
#include <iostream>
#include <string_view>
#include <functional>
#include <boost/program_options.hpp>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <Parsers/ParserQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/obfuscateQueries.h>
#include <Common/TerminalSize.h>
#include <Interpreters/Context.h>
#include <Functions/FunctionFactory.h>
#include <Functions/registerFunctions.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/registerTableFunctions.h>
#include <Storages/StorageFactory.h>
#include <Storages/registerStorages.h>
#include <DataTypes/DataTypeFactory.h>
#pragma GCC diagnostic ignored "-Wunused-function"
#pragma GCC diagnostic ignored "-Wmissing-declarations"
@ -22,6 +38,8 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
("oneline", "format in single line")
("quiet,q", "just check syntax, no output on success")
("multiquery,n", "allow multiple queries in the same file")
("obfuscate", "obfuscate instead of formatting")
("seed", po::value<std::string>(), "seed (arbitrary string) that determines the result of obfuscation")
;
boost::program_options::variables_map options;
@ -40,10 +58,17 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
bool oneline = options.count("oneline");
bool quiet = options.count("quiet");
bool multiple = options.count("multiquery");
bool obfuscate = options.count("obfuscate");
if (quiet && (hilite || oneline))
if (quiet && (hilite || oneline || obfuscate))
{
std::cerr << "Options 'hilite' or 'oneline' have no sense in 'quiet' mode." << std::endl;
std::cerr << "Options 'hilite' or 'oneline' or 'obfuscate' have no sense in 'quiet' mode." << std::endl;
return 2;
}
if (obfuscate && (hilite || oneline || quiet))
{
std::cerr << "Options 'hilite' or 'oneline' or 'quiet' have no sense in 'obfuscate' mode." << std::endl;
return 2;
}
@ -51,21 +76,66 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
ReadBufferFromFileDescriptor in(STDIN_FILENO);
readStringUntilEOF(query, in);
const char * pos = query.data();
const char * end = pos + query.size();
ParserQuery parser(end);
do
if (obfuscate)
{
ASTPtr res = parseQueryAndMovePosition(parser, pos, end, "query", multiple, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
if (!quiet)
WordMap obfuscated_words_map;
WordSet used_nouns;
SipHash hash_func;
if (options.count("seed"))
{
formatAST(*res, std::cout, hilite, oneline);
if (multiple)
std::cout << "\n;\n";
std::cout << std::endl;
std::string seed;
hash_func.update(options["seed"].as<std::string>());
}
} while (multiple && pos != end);
SharedContextHolder shared_context = Context::createShared();
Context context = Context::createGlobal(shared_context.get());
context.makeGlobalContext();
registerFunctions();
registerAggregateFunctions();
registerTableFunctions();
registerStorages();
std::unordered_set<std::string> additional_names;
auto all_known_storage_names = StorageFactory::instance().getAllRegisteredNames();
auto all_known_data_type_names = DataTypeFactory::instance().getAllRegisteredNames();
additional_names.insert(all_known_storage_names.begin(), all_known_storage_names.end());
additional_names.insert(all_known_data_type_names.begin(), all_known_data_type_names.end());
KnownIdentifierFunc is_known_identifier = [&](std::string_view name)
{
std::string what(name);
return FunctionFactory::instance().tryGet(what, context) != nullptr
|| AggregateFunctionFactory::instance().isAggregateFunctionName(what)
|| TableFunctionFactory::instance().isTableFunctionName(what)
|| additional_names.count(what);
};
WriteBufferFromFileDescriptor out(STDOUT_FILENO);
obfuscateQueries(query, out, obfuscated_words_map, used_nouns, hash_func, is_known_identifier);
}
else
{
const char * pos = query.data();
const char * end = pos + query.size();
ParserQuery parser(end);
do
{
ASTPtr res = parseQueryAndMovePosition(parser, pos, end, "query", multiple, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
if (!quiet)
{
formatAST(*res, std::cout, hilite, oneline);
if (multiple)
std::cout << "\n;\n";
std::cout << std::endl;
}
} while (multiple && pos != end);
}
}
catch (...)
{

View File

@ -3,7 +3,7 @@ set(CLICKHOUSE_SERVER_SOURCES
Server.cpp
)
if (OS_LINUX AND ARCH_AMD64)
if (OS_LINUX)
set (LINK_CONFIG_LIB INTERFACE "-Wl,${WHOLE_ARCHIVE} $<TARGET_FILE:clickhouse_server_configs> -Wl,${NO_WHOLE_ARCHIVE}")
endif ()

View File

@ -339,16 +339,23 @@ int Server::main(const std::vector<std::string> & /*args*/)
{
if (hasLinuxCapability(CAP_IPC_LOCK))
{
/// Get the memory area with (current) code segment.
/// It's better to lock only the code segment instead of calling "mlockall",
/// because otherwise debug info will be also locked in memory, and it can be huge.
auto [addr, len] = getMappedArea(reinterpret_cast<void *>(mainEntryClickHouseServer));
try
{
/// Get the memory area with (current) code segment.
/// It's better to lock only the code segment instead of calling "mlockall",
/// because otherwise debug info will be also locked in memory, and it can be huge.
auto [addr, len] = getMappedArea(reinterpret_cast<void *>(mainEntryClickHouseServer));
LOG_TRACE(log, "Will do mlock to prevent executable memory from being paged out. It may take a few seconds.");
if (0 != mlock(addr, len))
LOG_WARNING(log, "Failed mlock: {}", errnoToString(ErrorCodes::SYSTEM_ERROR));
else
LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed, total {}", ReadableSize(len));
LOG_TRACE(log, "Will do mlock to prevent executable memory from being paged out. It may take a few seconds.");
if (0 != mlock(addr, len))
LOG_WARNING(log, "Failed mlock: {}", errnoToString(ErrorCodes::SYSTEM_ERROR));
else
LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed, total {}", ReadableSize(len));
}
catch (...)
{
LOG_WARNING(log, "Cannot mlock: {}", getCurrentExceptionMessage(false));
}
}
else
{
@ -664,6 +671,10 @@ int Server::main(const std::vector<std::string> & /*args*/)
total_memory_tracker.setDescription("(total)");
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
/// Set current database name before loading tables and databases because
/// system logs may copy global context.
global_context->setCurrentDatabaseNameInGlobalContext(default_database);
LOG_INFO(log, "Loading metadata from {}", path);
try
@ -671,11 +682,14 @@ int Server::main(const std::vector<std::string> & /*args*/)
loadMetadataSystem(*global_context);
/// After attaching system databases we can initialize system log.
global_context->initializeSystemLogs();
auto & database_catalog = DatabaseCatalog::instance();
/// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
attachSystemTablesServer(*DatabaseCatalog::instance().getSystemDatabase(), has_zookeeper);
attachSystemTablesServer(*database_catalog.getSystemDatabase(), has_zookeeper);
/// Then, load remaining databases
loadMetadata(*global_context, default_database);
DatabaseCatalog::instance().loadDatabases();
database_catalog.loadDatabases();
/// After loading validate that default database exists
database_catalog.assertDatabaseExists(default_database);
}
catch (...)
{
@ -738,8 +752,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created"
" (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe).");
global_context->setCurrentDatabase(default_database);
if (has_zookeeper && config().has("distributed_ddl"))
{
/// DDL worker should be started after all tables were loaded

10
release
View File

@ -95,9 +95,9 @@ then
exit 3
fi
export DEB_CC=${DEB_CC=clang-6.0}
export DEB_CXX=${DEB_CXX=clang++-6.0}
EXTRAPACKAGES="$EXTRAPACKAGES clang-6.0 lld-6.0"
export DEB_CC=${DEB_CC=clang-10}
export DEB_CXX=${DEB_CXX=clang++-10}
EXTRAPACKAGES="$EXTRAPACKAGES clang-10 lld-10"
elif [[ $BUILD_TYPE == 'valgrind' ]]; then
MALLOC_OPTS="-DENABLE_TCMALLOC=0 -DENABLE_JEMALLOC=0"
VERSION_POSTFIX+="+valgrind"
@ -118,8 +118,8 @@ echo -e "\nCurrent version is $VERSION_STRING"
if [ -z "$NO_BUILD" ] ; then
gen_changelog "$VERSION_STRING" "" "$AUTHOR" ""
if [ -z "$USE_PBUILDER" ] ; then
DEB_CC=${DEB_CC:=`which gcc-9 gcc-8 gcc | head -n1`}
DEB_CXX=${DEB_CXX:=`which g++-9 g++-8 g++ | head -n1`}
DEB_CC=${DEB_CC:=`which gcc-10 gcc-9 gcc | head -n1`}
DEB_CXX=${DEB_CXX:=`which gcc-10 g++-9 g++ | head -n1`}
# Build (only binary packages).
debuild --preserve-env -e PATH \
-e DEB_CC=$DEB_CC -e DEB_CXX=$DEB_CXX -e CMAKE_FLAGS="$CMAKE_FLAGS" \

View File

@ -192,7 +192,7 @@ namespace
}
std::vector<AccessEntityPtr> parseUsers(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log)
std::vector<AccessEntityPtr> parseUsers(const Poco::Util::AbstractConfiguration & config)
{
Poco::Util::AbstractConfiguration::Keys user_names;
config.keys("users", user_names);
@ -200,16 +200,8 @@ namespace
std::vector<AccessEntityPtr> users;
users.reserve(user_names.size());
for (const auto & user_name : user_names)
{
try
{
users.push_back(parseUser(config, user_name));
}
catch (...)
{
tryLogCurrentException(log, "Could not parse user " + backQuote(user_name));
}
}
users.push_back(parseUser(config, user_name));
return users;
}
@ -256,12 +248,11 @@ namespace
}
quota->to_roles.add(user_ids);
return quota;
}
std::vector<AccessEntityPtr> parseQuotas(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log)
std::vector<AccessEntityPtr> parseQuotas(const Poco::Util::AbstractConfiguration & config)
{
Poco::Util::AbstractConfiguration::Keys user_names;
config.keys("users", user_names);
@ -278,76 +269,63 @@ namespace
quotas.reserve(quota_names.size());
for (const auto & quota_name : quota_names)
{
try
{
auto it = quota_to_user_ids.find(quota_name);
const std::vector<UUID> & quota_users = (it != quota_to_user_ids.end()) ? std::move(it->second) : std::vector<UUID>{};
quotas.push_back(parseQuota(config, quota_name, quota_users));
}
catch (...)
{
tryLogCurrentException(log, "Could not parse quota " + backQuote(quota_name));
}
auto it = quota_to_user_ids.find(quota_name);
const std::vector<UUID> & quota_users = (it != quota_to_user_ids.end()) ? std::move(it->second) : std::vector<UUID>{};
quotas.push_back(parseQuota(config, quota_name, quota_users));
}
return quotas;
}
std::vector<AccessEntityPtr> parseRowPolicies(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log)
std::vector<AccessEntityPtr> parseRowPolicies(const Poco::Util::AbstractConfiguration & config)
{
std::map<std::pair<String /* database */, String /* table */>, std::unordered_map<String /* user */, String /* filter */>> all_filters_map;
Poco::Util::AbstractConfiguration::Keys user_names;
config.keys("users", user_names);
try
for (const String & user_name : user_names)
{
config.keys("users", user_names);
for (const String & user_name : user_names)
const String databases_config = "users." + user_name + ".databases";
if (config.has(databases_config))
{
const String databases_config = "users." + user_name + ".databases";
if (config.has(databases_config))
Poco::Util::AbstractConfiguration::Keys database_keys;
config.keys(databases_config, database_keys);
/// Read tables within databases
for (const String & database_key : database_keys)
{
Poco::Util::AbstractConfiguration::Keys database_keys;
config.keys(databases_config, database_keys);
const String database_config = databases_config + "." + database_key;
/// Read tables within databases
for (const String & database_key : database_keys)
String database_name;
if (((database_key == "database") || (database_key.starts_with("database["))) && config.has(database_config + "[@name]"))
database_name = config.getString(database_config + "[@name]");
else if (size_t bracket_pos = database_key.find('['); bracket_pos != std::string::npos)
database_name = database_key.substr(0, bracket_pos);
else
database_name = database_key;
Poco::Util::AbstractConfiguration::Keys table_keys;
config.keys(database_config, table_keys);
/// Read table properties
for (const String & table_key : table_keys)
{
const String database_config = databases_config + "." + database_key;
String database_name;
if (((database_key == "database") || (database_key.starts_with("database["))) && config.has(database_config + "[@name]"))
database_name = config.getString(database_config + "[@name]");
else if (size_t bracket_pos = database_key.find('['); bracket_pos != std::string::npos)
database_name = database_key.substr(0, bracket_pos);
String table_config = database_config + "." + table_key;
String table_name;
if (((table_key == "table") || (table_key.starts_with("table["))) && config.has(table_config + "[@name]"))
table_name = config.getString(table_config + "[@name]");
else if (size_t bracket_pos = table_key.find('['); bracket_pos != std::string::npos)
table_name = table_key.substr(0, bracket_pos);
else
database_name = database_key;
table_name = table_key;
Poco::Util::AbstractConfiguration::Keys table_keys;
config.keys(database_config, table_keys);
/// Read table properties
for (const String & table_key : table_keys)
{
String table_config = database_config + "." + table_key;
String table_name;
if (((table_key == "table") || (table_key.starts_with("table["))) && config.has(table_config + "[@name]"))
table_name = config.getString(table_config + "[@name]");
else if (size_t bracket_pos = table_key.find('['); bracket_pos != std::string::npos)
table_name = table_key.substr(0, bracket_pos);
else
table_name = table_key;
String filter_config = table_config + ".filter";
all_filters_map[{database_name, table_name}][user_name] = config.getString(filter_config);
}
String filter_config = table_config + ".filter";
all_filters_map[{database_name, table_name}][user_name] = config.getString(filter_config);
}
}
}
}
catch (...)
{
tryLogCurrentException(log, "Could not parse row policies");
}
std::vector<AccessEntityPtr> policies;
for (auto & [database_and_table_name, user_to_filters] : all_filters_map)
@ -450,23 +428,14 @@ namespace
std::vector<AccessEntityPtr> parseSettingsProfiles(
const Poco::Util::AbstractConfiguration & config,
const std::function<void(const std::string_view &)> & check_setting_name_function,
Poco::Logger * log)
const std::function<void(const std::string_view &)> & check_setting_name_function)
{
std::vector<AccessEntityPtr> profiles;
Poco::Util::AbstractConfiguration::Keys profile_names;
config.keys("profiles", profile_names);
for (const auto & profile_name : profile_names)
{
try
{
profiles.push_back(parseSettingsProfile(config, profile_name, check_setting_name_function));
}
catch (...)
{
tryLogCurrentException(log, "Could not parse profile " + backQuote(profile_name));
}
}
profiles.push_back(parseSettingsProfile(config, profile_name, check_setting_name_function));
return profiles;
}
}
@ -520,13 +489,13 @@ void UsersConfigAccessStorage::setConfig(const Poco::Util::AbstractConfiguration
void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfiguration & config)
{
std::vector<std::pair<UUID, AccessEntityPtr>> all_entities;
for (const auto & entity : parseUsers(config, getLogger()))
for (const auto & entity : parseUsers(config))
all_entities.emplace_back(generateID(*entity), entity);
for (const auto & entity : parseQuotas(config, getLogger()))
for (const auto & entity : parseQuotas(config))
all_entities.emplace_back(generateID(*entity), entity);
for (const auto & entity : parseRowPolicies(config, getLogger()))
for (const auto & entity : parseRowPolicies(config))
all_entities.emplace_back(generateID(*entity), entity);
for (const auto & entity : parseSettingsProfiles(config, check_setting_name_function, getLogger()))
for (const auto & entity : parseSettingsProfiles(config, check_setting_name_function))
all_entities.emplace_back(generateID(*entity), entity);
memory_storage.setAll(all_entities);
}

View File

@ -1,22 +1,12 @@
#pragma once
#include <cstddef>
#include <cstdint>
#include <cassert>
#include <type_traits>
#include <common/defines.h>
/** Returns log2 of number, rounded down.
* Compiles to single 'bsr' instruction on x86.
* For zero argument, result is unspecified.
*/
inline unsigned int bitScanReverse(unsigned int x)
{
assert(x != 0);
return sizeof(unsigned int) * 8 - 1 - __builtin_clz(x);
}
/** For zero argument, result is zero.
* For arguments with most significand bit set, result is n.
* For other arguments, returns value, rounded up to power of two.
@ -41,10 +31,9 @@ inline size_t roundUpToPowerOfTwoOrZero(size_t n)
template <typename T>
inline size_t getLeadingZeroBits(T x)
inline size_t getLeadingZeroBitsUnsafe(T x)
{
if (!x)
return sizeof(x) * 8;
assert(x != 0);
if constexpr (sizeof(T) <= sizeof(unsigned int))
{
@ -60,10 +49,32 @@ inline size_t getLeadingZeroBits(T x)
}
}
template <typename T>
inline size_t getLeadingZeroBits(T x)
{
if (!x)
return sizeof(x) * 8;
return getLeadingZeroBitsUnsafe(x);
}
/** Returns log2 of number, rounded down.
* Compiles to single 'bsr' instruction on x86.
* For zero argument, result is unspecified.
*/
template <typename T>
inline uint32_t bitScanReverse(T x)
{
return (std::max<size_t>(sizeof(T), sizeof(unsigned int))) * 8 - 1 - getLeadingZeroBitsUnsafe(x);
}
// Unsafe since __builtin_ctz()-family explicitly state that result is undefined on x == 0
template <typename T>
inline size_t getTrailingZeroBitsUnsafe(T x)
{
assert(x != 0);
if constexpr (sizeof(T) <= sizeof(unsigned int))
{
return __builtin_ctz(x);
@ -88,8 +99,8 @@ inline size_t getTrailingZeroBits(T x)
}
/** Returns a mask that has '1' for `bits` LSB set:
* maskLowBits<UInt8>(3) => 00000111
*/
* maskLowBits<UInt8>(3) => 00000111
*/
template <typename T>
inline T maskLowBits(unsigned char bits)
{

View File

@ -138,6 +138,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
if (throw_on_error)
throw;
tryLogCurrentException(log, "Error updating configuration from '" + path + "' config.");
return;
}
LOG_DEBUG(log, "Loaded config '{}', performed update on configuration", path);

View File

@ -41,11 +41,6 @@ void FileChecker::setEmpty(const String & full_file_path)
map[fileName(full_file_path)] = 0;
}
FileChecker::Map FileChecker::getFileSizes() const
{
return map;
}
CheckResults FileChecker::check() const
{
// Read the files again every time you call `check` - so as not to violate the constancy.

View File

@ -27,12 +27,10 @@ public:
/// The purpose of this function is to rollback a group of unfinished writes.
void repair();
private:
/// File name -> size.
using Map = std::map<String, UInt64>;
Map getFileSizes() const;
private:
void initialize();
void updateImpl(const String & file_path);
void load(Map & local_map, const String & path) const;

View File

@ -23,18 +23,15 @@ Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String &
}
String Macros::expand(const String & s,
size_t level,
const String & database_name,
const String & table_name,
const UUID & uuid) const
MacroExpansionInfo & info) const
{
if (s.find('{') == String::npos)
return s;
if (level && s.size() > 65536)
if (info.level && s.size() > 65536)
throw Exception("Too long string while expanding macros", ErrorCodes::SYNTAX_ERROR);
if (level >= 10)
if (info.level >= 10)
throw Exception("Too deep recursion while expanding macros: '" + s + "'", ErrorCodes::SYNTAX_ERROR);
String res;
@ -64,17 +61,28 @@ String Macros::expand(const String & s,
/// Prefer explicit macros over implicit.
if (it != macros.end())
res += it->second;
else if (macro_name == "database" && !database_name.empty())
res += database_name;
else if (macro_name == "table" && !table_name.empty())
res += table_name;
else if (macro_name == "database" && !info.database_name.empty())
res += info.database_name;
else if (macro_name == "table" && !info.table_name.empty())
res += info.table_name;
else if (macro_name == "uuid")
{
if (uuid == UUIDHelpers::Nil)
if (info.uuid == UUIDHelpers::Nil)
throw Exception("Macro 'uuid' and empty arguments of ReplicatedMergeTree "
"are supported only for ON CLUSTER queries with Atomic database engine",
ErrorCodes::SYNTAX_ERROR);
res += toString(uuid);
/// For ON CLUSTER queries we don't want to require all macros definitions in initiator's config.
/// However, initiator must check that for cross-replication cluster zookeeper_path does not contain {uuid} macro.
/// It becomes impossible to check if {uuid} is contained inside some unknown macro.
if (info.level)
throw Exception("Macro 'uuid' should not be inside another macro", ErrorCodes::SYNTAX_ERROR);
res += toString(info.uuid);
info.expanded_uuid = true;
}
else if (info.ignore_unknown)
{
res += macro_name;
info.has_unknown = true;
}
else
throw Exception("No macro '" + macro_name +
@ -84,7 +92,8 @@ String Macros::expand(const String & s,
pos = end + 1;
}
return expand(res, level + 1, database_name, table_name);
++info.level;
return expand(res, info);
}
String Macros::getValue(const String & key) const
@ -94,9 +103,20 @@ String Macros::getValue(const String & key) const
throw Exception("No macro " + key + " in config", ErrorCodes::SYNTAX_ERROR);
}
String Macros::expand(const String & s) const
{
MacroExpansionInfo info;
return expand(s, info);
}
String Macros::expand(const String & s, const StorageID & table_id, bool allow_uuid) const
{
return expand(s, 0, table_id.database_name, table_id.table_name, allow_uuid ? table_id.uuid : UUIDHelpers::Nil);
MacroExpansionInfo info;
info.database_name = table_id.database_name;
info.table_name = table_id.table_name;
info.uuid = allow_uuid ? table_id.uuid : UUIDHelpers::Nil;
return expand(s, info);
}
Names Macros::expand(const Names & source_names, size_t level) const
@ -104,8 +124,12 @@ Names Macros::expand(const Names & source_names, size_t level) const
Names result_names;
result_names.reserve(source_names.size());
MacroExpansionInfo info;
for (const String & name : source_names)
result_names.push_back(expand(name, level));
{
info.level = level;
result_names.push_back(expand(name, info));
}
return result_names;
}

View File

@ -27,15 +27,28 @@ public:
Macros() = default;
Macros(const Poco::Util::AbstractConfiguration & config, const String & key);
struct MacroExpansionInfo
{
/// Settings
String database_name;
String table_name;
UUID uuid = UUIDHelpers::Nil;
bool ignore_unknown = false;
/// Information about macro expansion
size_t level = 0;
bool expanded_uuid = false;
bool has_unknown = false;
};
/** Replace the substring of the form {macro_name} with the value for macro_name, obtained from the config file.
* If {database} and {table} macros aren`t defined explicitly, expand them as database_name and table_name respectively.
* level - the level of recursion.
*/
String expand(const String & s,
size_t level = 0,
const String & database_name = "",
const String & table_name = "",
const UUID & uuid = UUIDHelpers::Nil) const;
MacroExpansionInfo & info) const;
String expand(const String & s) const;
String expand(const String & s, const StorageID & table_id, bool allow_uuid) const;

View File

@ -67,10 +67,19 @@ inline bool isASCII(char c)
return static_cast<unsigned char>(c) < 0x80;
}
inline bool isLowerAlphaASCII(char c)
{
return (c >= 'a' && c <= 'z');
}
inline bool isUpperAlphaASCII(char c)
{
return (c >= 'A' && c <= 'Z');
}
inline bool isAlphaASCII(char c)
{
return (c >= 'a' && c <= 'z')
|| (c >= 'A' && c <= 'Z');
return isLowerAlphaASCII(c) || isUpperAlphaASCII(c);
}
inline bool isNumericASCII(char c)
@ -122,6 +131,16 @@ inline bool isPrintableASCII(char c)
return uc >= 32 && uc <= 126; /// 127 is ASCII DEL.
}
inline bool isPunctuationASCII(char c)
{
uint8_t uc = c;
return (uc >= 33 && uc <= 47)
|| (uc >= 58 && uc <= 64)
|| (uc >= 91 && uc <= 96)
|| (uc >= 123 && uc <= 125);
}
inline bool isValidIdentifier(const std::string_view & str)
{
return !str.empty() && isValidIdentifierBegin(str[0]) && std::all_of(str.begin() + 1, str.end(), isWordCharASCII);

View File

@ -4,6 +4,7 @@
#include <Common/Exception.h>
#include <Common/randomSeed.h>
#include <Common/SipHash.h>
#include <common/getThreadId.h>
#include <common/types.h>
@ -19,7 +20,7 @@ namespace DB
DB::UInt64 randomSeed()
{
struct timespec times;
if (clock_gettime(CLOCK_THREAD_CPUTIME_ID, &times))
if (clock_gettime(CLOCK_MONOTONIC, &times))
DB::throwFromErrno("Cannot clock_gettime.", DB::ErrorCodes::CANNOT_CLOCK_GETTIME);
/// Not cryptographically secure as time, pid and stack address can be predictable.
@ -27,7 +28,7 @@ DB::UInt64 randomSeed()
SipHash hash;
hash.update(times.tv_nsec);
hash.update(times.tv_sec);
hash.update(getpid());
hash.update(getThreadId());
hash.update(&times);
return hash.get64();
}

View File

@ -350,7 +350,7 @@ class IColumn;
M(UInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.", 0) \
M(UInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \
\
M(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \
M(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic, "Default database engine.", 0) \
M(Bool, show_table_uuid_in_table_create_query_if_not_nil, false, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \
M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \
M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \

View File

@ -926,7 +926,7 @@ void CacheDictionary::update(UpdateUnitPtr & update_unit_ptr) const
else
cell.setExpiresAt(std::chrono::time_point<std::chrono::system_clock>::max());
update_unit_ptr->getPresentIdHandler()(id, cell_idx);
update_unit_ptr->callPresentIdHandler(id, cell_idx);
/// mark corresponding id as found
remaining_ids[id] = 1;
}
@ -988,9 +988,9 @@ void CacheDictionary::update(UpdateUnitPtr & update_unit_ptr) const
if (was_default)
cell.setDefault();
if (was_default)
update_unit_ptr->getAbsentIdHandler()(id, cell_idx);
update_unit_ptr->callAbsentIdHandler(id, cell_idx);
else
update_unit_ptr->getPresentIdHandler()(id, cell_idx);
update_unit_ptr->callPresentIdHandler(id, cell_idx);
continue;
}
/// We don't have expired data for that `id` so all we can do is to rethrow `last_exception`.
@ -1022,7 +1022,7 @@ void CacheDictionary::update(UpdateUnitPtr & update_unit_ptr) const
setDefaultAttributeValue(attribute, cell_idx);
/// inform caller that the cell has not been found
update_unit_ptr->getAbsentIdHandler()(id, cell_idx);
update_unit_ptr->callAbsentIdHandler(id, cell_idx);
}
ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedMiss, not_found_num);

View File

@ -399,16 +399,18 @@ private:
absent_id_handler([](Key, size_t){}){}
PresentIdHandler getPresentIdHandler()
void callPresentIdHandler(Key key, size_t cell_idx)
{
std::lock_guard lock(callback_mutex);
return can_use_callback ? present_id_handler : PresentIdHandler{};
if (can_use_callback)
present_id_handler(key, cell_idx);
}
AbsentIdHandler getAbsentIdHandler()
void callAbsentIdHandler(Key key, size_t cell_idx)
{
std::lock_guard lock(callback_mutex);
return can_use_callback ? absent_id_handler : AbsentIdHandler{};
if (can_use_callback)
absent_id_handler(key, cell_idx);
}
std::vector<Key> requested_ids;

View File

@ -148,7 +148,9 @@ void CacheDictionary::getItemsNumberImpl(
std::begin(cache_expired_ids), std::end(cache_expired_ids),
std::back_inserter(required_ids), [](auto & pair) { return pair.first; });
auto on_cell_updated = [&] (const auto id, const auto cell_idx)
auto on_cell_updated =
[&attribute_array, &cache_not_found_ids, &cache_expired_ids, &out]
(const auto id, const auto cell_idx)
{
const auto attribute_value = attribute_array[cell_idx];

View File

@ -23,13 +23,27 @@ namespace ErrorCodes
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
}
MySQLBlockInputStream::Connection::Connection(
const mysqlxx::PoolWithFailover::Entry & entry_,
const std::string & query_str)
: entry(entry_)
, query{entry->query(query_str)}
, result{query.use()}
{
}
MySQLBlockInputStream::MySQLBlockInputStream(
const mysqlxx::PoolWithFailover::Entry & entry_, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_, const bool auto_close_)
: entry{entry_}, query{this->entry->query(query_str)}, result{query.use()}, max_block_size{max_block_size_}, auto_close{auto_close_}
const mysqlxx::PoolWithFailover::Entry & entry,
const std::string & query_str,
const Block & sample_block,
const UInt64 max_block_size_,
const bool auto_close_)
: connection{std::make_unique<Connection>(entry, query_str)}
, max_block_size{max_block_size_}
, auto_close{auto_close_}
{
if (sample_block.columns() != result.getNumFields())
throw Exception{"mysqlxx::UseQueryResult contains " + toString(result.getNumFields()) + " columns while "
if (sample_block.columns() != connection->result.getNumFields())
throw Exception{"mysqlxx::UseQueryResult contains " + toString(connection->result.getNumFields()) + " columns while "
+ toString(sample_block.columns()) + " expected",
ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH};
@ -106,11 +120,11 @@ namespace
Block MySQLBlockInputStream::readImpl()
{
auto row = result.fetch();
auto row = connection->result.fetch();
if (!row)
{
if (auto_close)
entry.disconnect();
connection->entry.disconnect();
return {};
}
@ -145,11 +159,42 @@ Block MySQLBlockInputStream::readImpl()
if (num_rows == max_block_size)
break;
row = result.fetch();
row = connection->result.fetch();
}
return description.sample_block.cloneWithColumns(std::move(columns));
}
MySQLBlockInputStream::MySQLBlockInputStream(
const Block & sample_block_,
UInt64 max_block_size_,
bool auto_close_)
: max_block_size(max_block_size_)
, auto_close(auto_close_)
{
description.init(sample_block_);
}
MySQLLazyBlockInputStream::MySQLLazyBlockInputStream(
mysqlxx::Pool & pool_,
const std::string & query_str_,
const Block & sample_block_,
const UInt64 max_block_size_,
const bool auto_close_)
: MySQLBlockInputStream(sample_block_, max_block_size_, auto_close_)
, pool(pool_)
, query_str(query_str_)
{
}
void MySQLLazyBlockInputStream::readPrefix()
{
connection = std::make_unique<Connection>(pool.get(), query_str);
if (description.sample_block.columns() != connection->result.getNumFields())
throw Exception{"mysqlxx::UseQueryResult contains " + toString(connection->result.getNumFields()) + " columns while "
+ toString(description.sample_block.columns()) + " expected",
ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH};
}
}
#endif

View File

@ -10,12 +10,13 @@
namespace DB
{
/// Allows processing results of a MySQL query as a sequence of Blocks, simplifies chaining
class MySQLBlockInputStream final : public IBlockInputStream
class MySQLBlockInputStream : public IBlockInputStream
{
public:
MySQLBlockInputStream(
const mysqlxx::PoolWithFailover::Entry & entry_,
const mysqlxx::PoolWithFailover::Entry & entry,
const std::string & query_str,
const Block & sample_block,
const UInt64 max_block_size_,
@ -25,15 +26,43 @@ public:
Block getHeader() const override { return description.sample_block.cloneEmpty(); }
private:
protected:
MySQLBlockInputStream(const Block & sample_block_, UInt64 max_block_size_, bool auto_close_);
Block readImpl() override;
mysqlxx::PoolWithFailover::Entry entry;
mysqlxx::Query query;
mysqlxx::UseQueryResult result;
struct Connection
{
Connection(const mysqlxx::PoolWithFailover::Entry & entry_, const std::string & query_str);
mysqlxx::PoolWithFailover::Entry entry;
mysqlxx::Query query;
mysqlxx::UseQueryResult result;
};
std::unique_ptr<Connection> connection;
const UInt64 max_block_size;
const bool auto_close;
ExternalResultDescription description;
};
/// Like MySQLBlockInputStream, but allocates connection only when reading is starting.
/// It allows to create a lot of stream objects without occupation of all connection pool.
class MySQLLazyBlockInputStream final : public MySQLBlockInputStream
{
public:
MySQLLazyBlockInputStream(
mysqlxx::Pool & pool_,
const std::string & query_str_,
const Block & sample_block_,
const UInt64 max_block_size_,
const bool auto_close_ = false);
private:
void readPrefix() override;
mysqlxx::Pool & pool;
std::string query_str;
};
}

View File

@ -80,7 +80,7 @@ public:
DataTypePtr getReturnType(const ColumnsWithTypeAndName &) const override { return {}; } // Not used
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return true; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }

View File

@ -235,11 +235,7 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, co
return header.cloneWithColumns(std::move(columns));
}
/** Create a block for set from literal.
* 'set_element_types' - types of what are on the left hand side of IN.
* 'right_arg' - Literal - Tuple or Array.
*/
static Block createBlockForSet(
Block createBlockForSet(
const DataTypePtr & left_arg_type,
const ASTPtr & right_arg,
const DataTypes & set_element_types,
@ -280,14 +276,7 @@ static Block createBlockForSet(
return block;
}
/** Create a block for set from expression.
* 'set_element_types' - types of what are on the left hand side of IN.
* 'right_arg' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6).
*
* We need special implementation for ASTFunction, because in case, when we interpret
* large tuple or array as function, `evaluateConstantExpression` works extremely slow.
*/
static Block createBlockForSet(
Block createBlockForSet(
const DataTypePtr & left_arg_type,
const std::shared_ptr<ASTFunction> & right_arg,
const DataTypes & set_element_types,

View File

@ -16,11 +16,37 @@ struct ExpressionAction;
class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
/// The case of an explicit enumeration of values.
/// The case of an explicit enumeration of values.
SetPtr makeExplicitSet(
const ASTFunction * node, const Block & sample_block, bool create_ordered_set,
const Context & context, const SizeLimits & limits, PreparedSets & prepared_sets);
/** Create a block for set from expression.
* 'set_element_types' - types of what are on the left hand side of IN.
* 'right_arg' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6).
*
* We need special implementation for ASTFunction, because in case, when we interpret
* large tuple or array as function, `evaluateConstantExpression` works extremely slow.
*
* Note: this and following functions are used in third-party applications in Arcadia, so
* they should be declared in header file.
*
*/
Block createBlockForSet(
const DataTypePtr & left_arg_type,
const std::shared_ptr<ASTFunction> & right_arg,
const DataTypes & set_element_types,
const Context & context);
/** Create a block for set from literal.
* 'set_element_types' - types of what are on the left hand side of IN.
* 'right_arg' - Literal - Tuple or Array.
*/
Block createBlockForSet(
const DataTypePtr & left_arg_type,
const ASTPtr & right_arg,
const DataTypes & set_element_types,
const Context & context);
/** For ActionsVisitor
* A stack of ExpressionActions corresponding to nested lambda expressions.

View File

@ -623,4 +623,21 @@ const std::string & Cluster::ShardInfo::pathForInsert(bool prefer_localhost_repl
return dir_name_for_internal_replication_with_local;
}
bool Cluster::maybeCrossReplication() const
{
/// Cluster can be used for cross-replication if some replicas have different default database names,
/// so one clickhouse-server instance can contain multiple replicas.
if (addresses_with_failover.empty())
return false;
const String & database_name = addresses_with_failover.front().front().default_database;
for (const auto & shard : addresses_with_failover)
for (const auto & replica : shard)
if (replica.default_database != database_name)
return true;
return false;
}
}

View File

@ -193,6 +193,10 @@ public:
/// Get a new Cluster that contains all servers (all shards with all replicas) from existing cluster as independent shards.
std::unique_ptr<Cluster> getClusterWithReplicasAsShards(const Settings & settings) const;
/// Returns false if cluster configuration doesn't allow to use it for cross-replication.
/// NOTE: true does not mean, that it's actually a cross-replication cluster.
bool maybeCrossReplication() const;
private:
using SlotToShard = std::vector<UInt64>;
SlotToShard slot_to_shard;

View File

@ -968,6 +968,7 @@ StoragePtr Context::getViewSource()
Settings Context::getSettings() const
{
auto lock = getLock();
return settings;
}
@ -1088,6 +1089,18 @@ String Context::getInitialQueryId() const
}
void Context::setCurrentDatabaseNameInGlobalContext(const String & name)
{
if (global_context != this)
throw Exception("Cannot set current database for non global context, this method should be used during server initialization", ErrorCodes::LOGICAL_ERROR);
auto lock = getLock();
if (!current_database.empty())
throw Exception("Default database name cannot be changed in global context without server restart", ErrorCodes::LOGICAL_ERROR);
current_database = name;
}
void Context::setCurrentDatabase(const String & name)
{
DatabaseCatalog::instance().assertDatabaseExists(name);

View File

@ -359,6 +359,9 @@ public:
String getInitialQueryId() const;
void setCurrentDatabase(const String & name);
/// Set current_database for global context. We don't validate that database
/// exists because it should be set before databases loading.
void setCurrentDatabaseNameInGlobalContext(const String & name);
void setCurrentQueryId(const String & query_id);
void killCurrentQuery();

View File

@ -525,7 +525,7 @@ static bool allowDictJoin(StoragePtr joined_storage, const Context & context, St
if (!dict)
return false;
dict_name = dict->dictionaryName();
dict_name = dict->resolvedDictionaryName();
auto dictionary = context.getExternalDictionariesLoader().getDictionary(dict_name);
if (!dictionary)
return false;

View File

@ -5,6 +5,7 @@
#include <Common/StringUtils/StringUtils.h>
#include <Common/escapeForFileName.h>
#include <Common/typeid_cast.h>
#include <Common/Macros.h>
#include <Core/Defines.h>
#include <Core/Settings.h>
@ -853,17 +854,60 @@ BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create)
return {};
}
void InterpreterCreateQuery::prepareOnClusterQuery(ASTCreateQuery & create, const Context & context, const String & cluster_name)
{
if (create.attach)
return;
/// For CREATE query generate UUID on initiator, so it will be the same on all hosts.
/// It will be ignored if database does not support UUIDs.
if (create.uuid == UUIDHelpers::Nil)
create.uuid = UUIDHelpers::generateV4();
/// For cross-replication cluster we cannot use UUID in replica path.
String cluster_name_expanded = context.getMacros()->expand(cluster_name);
ClusterPtr cluster = context.getCluster(cluster_name_expanded);
if (cluster->maybeCrossReplication())
{
/// Check that {uuid} macro is not used in zookeeper_path for ReplicatedMergeTree.
/// Otherwise replicas will generate different paths.
if (!create.storage)
return;
if (!create.storage->engine)
return;
if (!startsWith(create.storage->engine->name, "Replicated"))
return;
bool has_explicit_zk_path_arg = create.storage->engine->arguments &&
create.storage->engine->arguments->children.size() >= 2 &&
create.storage->engine->arguments->children[0]->as<ASTLiteral>() &&
create.storage->engine->arguments->children[0]->as<ASTLiteral>()->value.getType() == Field::Types::String;
if (has_explicit_zk_path_arg)
{
String zk_path = create.storage->engine->arguments->children[0]->as<ASTLiteral>()->value.get<String>();
Macros::MacroExpansionInfo info;
info.uuid = create.uuid;
info.ignore_unknown = true;
context.getMacros()->expand(zk_path, info);
if (!info.expanded_uuid)
return;
}
throw Exception("Seems like cluster is configured for cross-replication, "
"but zookeeper_path for ReplicatedMergeTree is not specified or contains {uuid} macro. "
"It's not supported for cross replication, because tables must have different UUIDs. "
"Please specify unique zookeeper_path explicitly.", ErrorCodes::INCORRECT_QUERY);
}
}
BlockIO InterpreterCreateQuery::execute()
{
auto & create = query_ptr->as<ASTCreateQuery &>();
if (!create.cluster.empty())
{
/// Allows to execute ON CLUSTER queries during version upgrade
bool force_backward_compatibility = !context.getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil;
/// For CREATE query generate UUID on initiator, so it will be the same on all hosts.
/// It will be ignored if database does not support UUIDs.
if (!force_backward_compatibility && !create.attach && create.uuid == UUIDHelpers::Nil)
create.uuid = UUIDHelpers::generateV4();
prepareOnClusterQuery(create, context, create.cluster);
return executeDDLQueryOnCluster(query_ptr, context, getRequiredAccess());
}

View File

@ -55,6 +55,8 @@ public:
static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, const Context & context, bool sanity_check_compression_codecs);
static ConstraintsDescription getConstraintsDescription(const ASTExpressionList * constraints);
static void prepareOnClusterQuery(ASTCreateQuery & create, const Context & context, const String & cluster_name);
private:
struct TableProperties
{

View File

@ -401,7 +401,7 @@ void ProcessList::killAllQueries()
QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_events, bool get_settings) const
{
QueryStatusInfo res;
QueryStatusInfo res{};
res.query = query;
res.client_info = client_info;
@ -431,7 +431,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even
}
if (get_settings && query_context)
res.query_settings = std::make_shared<Settings>(query_context->getSettingsRef());
res.query_settings = std::make_shared<Settings>(query_context->getSettings());
return res;
}

View File

@ -438,7 +438,7 @@ void SystemLog<LogElement>::flushImpl(const std::vector<LogElement> & to_flush,
ASTPtr query_ptr(insert.release());
// we need query context to do inserts to target table with MV containing subqueries or joins
auto insert_context = Context(context);
Context insert_context(context);
insert_context.makeQueryContext();
InterpreterInsertQuery interpreter(query_ptr, insert_context);

View File

@ -53,7 +53,7 @@ public:
ASTPtr clone() const override
{
auto replacement = std::make_shared<Replacement>(*this);
replacement->name = name;
replacement->children.clear();
replacement->expr = expr->clone();
replacement->children.push_back(replacement->expr);
return replacement;

View File

@ -0,0 +1,937 @@
#include <cassert>
#include <Parsers/obfuscateQueries.h>
#include <Parsers/Lexer.h>
#include <Poco/String.h>
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/BitHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromMemory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_MANY_TEMPORARY_COLUMNS;
}
namespace
{
const std::unordered_set<std::string_view> keywords
{
"CREATE", "DATABASE", "IF", "NOT", "EXISTS", "TEMPORARY", "TABLE", "ON", "CLUSTER", "DEFAULT",
"MATERIALIZED", "ALIAS", "ENGINE", "AS", "VIEW", "POPULATE", "SETTINGS", "ATTACH", "DETACH", "DROP",
"RENAME", "TO", "ALTER", "ADD", "MODIFY", "CLEAR", "COLUMN", "AFTER", "COPY", "PROJECT",
"PRIMARY", "KEY", "CHECK", "PARTITION", "PART", "FREEZE", "FETCH", "FROM", "SHOW", "INTO",
"OUTFILE", "FORMAT", "TABLES", "DATABASES", "LIKE", "PROCESSLIST", "CASE", "WHEN", "THEN", "ELSE",
"END", "DESCRIBE", "DESC", "USE", "SET", "OPTIMIZE", "FINAL", "DEDUPLICATE", "INSERT", "VALUES",
"SELECT", "DISTINCT", "SAMPLE", "ARRAY", "JOIN", "GLOBAL", "LOCAL", "ANY", "ALL", "INNER",
"LEFT", "RIGHT", "FULL", "OUTER", "CROSS", "USING", "PREWHERE", "WHERE", "GROUP", "BY",
"WITH", "TOTALS", "HAVING", "ORDER", "COLLATE", "LIMIT", "UNION", "AND", "OR", "ASC",
"IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE",
"PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE",
"IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "RANDOMIZED",
"INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE", "DICTIONARY"
};
const std::unordered_set<std::string_view> keep_words
{
"id", "name", "value", "num",
"Id", "Name", "Value", "Num",
"ID", "NAME", "VALUE", "NUM",
};
/// The list of nouns collected from here: http://www.desiquintans.com/nounlist, Public domain.
std::initializer_list<std::string_view> nouns
{
"aardvark", "abacus", "abbey", "abbreviation", "abdomen", "ability", "abnormality", "abolishment", "abortion",
"abrogation", "absence", "abundance", "abuse", "academics", "academy", "accelerant", "accelerator", "accent", "acceptance", "access",
"accessory", "accident", "accommodation", "accompanist", "accomplishment", "accord", "accordance", "accordion", "account", "accountability",
"accountant", "accounting", "accuracy", "accusation", "acetate", "achievement", "achiever", "acid", "acknowledgment", "acorn", "acoustics",
"acquaintance", "acquisition", "acre", "acrylic", "act", "action", "activation", "activist", "activity", "actor", "actress", "acupuncture",
"ad", "adaptation", "adapter", "addiction", "addition", "address", "adjective", "adjustment", "admin", "administration", "administrator",
"admire", "admission", "adobe", "adoption", "adrenalin", "adrenaline", "adult", "adulthood", "advance", "advancement", "advantage", "advent",
"adverb", "advertisement", "advertising", "advice", "adviser", "advocacy", "advocate", "affair", "affect", "affidavit", "affiliate",
"affinity", "afoul", "afterlife", "aftermath", "afternoon", "aftershave", "aftershock", "afterthought", "age", "agency", "agenda", "agent",
"aggradation", "aggression", "aglet", "agony", "agreement", "agriculture", "aid", "aide", "aim", "air", "airbag", "airbus", "aircraft",
"airfare", "airfield", "airforce", "airline", "airmail", "airman", "airplane", "airport", "airship", "airspace", "alarm", "alb", "albatross",
"album", "alcohol", "alcove", "alder", "ale", "alert", "alfalfa", "algebra", "algorithm", "alias", "alibi", "alien", "allegation", "allergist",
"alley", "alliance", "alligator", "allocation", "allowance", "alloy", "alluvium", "almanac", "almighty", "almond", "alpaca", "alpenglow",
"alpenhorn", "alpha", "alphabet", "altar", "alteration", "alternative", "altitude", "alto", "aluminium", "aluminum", "amazement", "amazon",
"ambassador", "amber", "ambience", "ambiguity", "ambition", "ambulance", "amendment", "amenity", "ammunition", "amnesty", "amount", "amusement",
"anagram", "analgesia", "analog", "analogue", "analogy", "analysis", "analyst", "analytics", "anarchist", "anarchy", "anatomy", "ancestor",
"anchovy", "android", "anesthesiologist", "anesthesiology", "angel", "anger", "angina", "angiosperm", "angle", "angora", "angstrom",
"anguish", "animal", "anime", "anise", "ankle", "anklet", "anniversary", "announcement", "annual", "anorak", "answer", "ant", "anteater",
"antecedent", "antechamber", "antelope", "antennae", "anterior", "anthropology", "antibody", "anticipation", "anticodon", "antigen",
"antique", "antiquity", "antler", "antling", "anxiety", "anybody", "anyone", "anything", "anywhere", "apartment", "ape", "aperitif",
"apology", "app", "apparatus", "apparel", "appeal", "appearance", "appellation", "appendix", "appetiser", "appetite", "appetizer", "applause",
"apple", "applewood", "appliance", "application", "appointment", "appreciation", "apprehension", "approach", "appropriation", "approval",
"apricot", "apron", "apse", "aquarium", "aquifer", "arcade", "arch", "archaeologist", "archaeology", "archeology", "archer",
"architect", "architecture", "archives", "area", "arena", "argument", "arithmetic", "ark", "arm", "armadillo", "armament",
"armchair", "armoire", "armor", "armour", "armpit", "armrest", "army", "arrangement", "array", "arrest", "arrival", "arrogance", "arrow",
"art", "artery", "arthur", "artichoke", "article", "artifact", "artificer", "artist", "ascend", "ascent", "ascot", "ash", "ashram", "ashtray",
"aside", "asparagus", "aspect", "asphalt", "aspic", "assassination", "assault", "assembly", "assertion", "assessment", "asset",
"assignment", "assist", "assistance", "assistant", "associate", "association", "assumption", "assurance", "asterisk", "astrakhan", "astrolabe",
"astrologer", "astrology", "astronomy", "asymmetry", "atelier", "atheist", "athlete", "athletics", "atmosphere", "atom", "atrium", "attachment",
"attack", "attacker", "attainment", "attempt", "attendance", "attendant", "attention", "attenuation", "attic", "attitude", "attorney",
"attraction", "attribute", "auction", "audience", "audit", "auditorium", "aunt", "authentication", "authenticity", "author", "authorisation",
"authority", "authorization", "auto", "autoimmunity", "automation", "automaton", "autumn", "availability", "avalanche", "avenue", "average",
"avocado", "award", "awareness", "awe", "axis", "azimuth", "babe", "baboon", "babushka", "baby", "bachelor", "back", "backbone",
"backburn", "backdrop", "background", "backpack", "backup", "backyard", "bacon", "bacterium", "badge", "badger", "bafflement", "bag",
"bagel", "baggage", "baggie", "baggy", "bagpipe", "bail", "bait", "bake", "baker", "bakery", "bakeware", "balaclava", "balalaika", "balance",
"balcony", "ball", "ballet", "balloon", "balloonist", "ballot", "ballpark", "bamboo", "ban", "banana", "band", "bandana", "bandanna",
"bandolier", "bandwidth", "bangle", "banjo", "bank", "bankbook", "banker", "banking", "bankruptcy", "banner", "banquette", "banyan",
"baobab", "bar", "barbecue", "barbeque", "barber", "barbiturate", "bargain", "barge", "baritone", "barium", "bark", "barley", "barn",
"barometer", "barracks", "barrage", "barrel", "barrier", "barstool", "bartender", "base", "baseball", "baseboard", "baseline", "basement",
"basics", "basil", "basin", "basis", "basket", "basketball", "bass", "bassinet", "bassoon", "bat", "bath", "bather", "bathhouse", "bathrobe",
"bathroom", "bathtub", "battalion", "batter", "battery", "batting", "battle", "battleship", "bay", "bayou", "beach", "bead", "beak",
"beam", "bean", "beancurd", "beanie", "beanstalk", "bear", "beard", "beast", "beastie", "beat", "beating", "beauty", "beaver", "beck",
"bed", "bedrock", "bedroom", "bee", "beech", "beef", "beer", "beet", "beetle", "beggar", "beginner", "beginning", "begonia", "behalf",
"behavior", "behaviour", "beheading", "behest", "behold", "being", "belfry", "belief", "believer", "bell", "belligerency", "bellows",
"belly", "belt", "bench", "bend", "beneficiary", "benefit", "beret", "berry", "bestseller", "bet", "beverage", "beyond",
"bias", "bibliography", "bicycle", "bid", "bidder", "bidding", "bidet", "bifocals", "bijou", "bike", "bikini", "bill", "billboard", "billing",
"billion", "bin", "binoculars", "biology", "biopsy", "biosphere", "biplane", "birch", "bird", "birdbath", "birdcage",
"birdhouse", "birth", "birthday", "biscuit", "bit", "bite", "bitten", "bitter", "black", "blackberry", "blackbird", "blackboard", "blackfish",
"blackness", "bladder", "blade", "blame", "blank", "blanket", "blast", "blazer", "blend", "blessing", "blight", "blind", "blinker", "blister",
"blizzard", "block", "blocker", "blog", "blogger", "blood", "bloodflow", "bloom", "bloomer", "blossom", "blouse", "blow", "blowgun",
"blowhole", "blue", "blueberry", "blush", "boar", "board", "boat", "boatload", "boatyard", "bob", "bobcat", "body", "bog", "bolero",
"bolt", "bomb", "bomber", "bombing", "bond", "bonding", "bondsman", "bone", "bonfire", "bongo", "bonnet", "bonsai", "bonus", "boogeyman",
"book", "bookcase", "bookend", "booking", "booklet", "bookmark", "boolean", "boom", "boon", "boost", "booster", "boot", "bootee", "bootie",
"booty", "border", "bore", "borrower", "borrowing", "bosom", "boss", "botany", "bother", "bottle", "bottling", "bottom",
"boudoir", "bough", "boulder", "boulevard", "boundary", "bouquet", "bourgeoisie", "bout", "boutique", "bow", "bower", "bowl", "bowler",
"bowling", "bowtie", "box", "boxer", "boxspring", "boy", "boycott", "boyfriend", "boyhood", "boysenberry", "bra", "brace", "bracelet",
"bracket", "brain", "brake", "bran", "branch", "brand", "brandy", "brass", "brassiere", "bratwurst", "bread", "breadcrumb", "breadfruit",
"break", "breakdown", "breakfast", "breakpoint", "breakthrough", "breast", "breastplate", "breath", "breeze", "brewer", "bribery", "brick",
"bricklaying", "bride", "bridge", "brief", "briefing", "briefly", "briefs", "brilliant", "brink", "brisket", "broad", "broadcast", "broccoli",
"brochure", "brocolli", "broiler", "broker", "bronchitis", "bronco", "bronze", "brooch", "brood", "brook", "broom", "brother",
"brow", "brown", "brownie", "browser", "browsing", "brunch", "brush", "brushfire", "brushing", "bubble", "buck", "bucket", "buckle",
"buckwheat", "bud", "buddy", "budget", "buffalo", "buffer", "buffet", "bug", "buggy", "bugle", "builder", "building", "bulb", "bulk",
"bull", "bulldozer", "bullet", "bump", "bumper", "bun", "bunch", "bungalow", "bunghole", "bunkhouse", "burden", "bureau",
"burglar", "burial", "burlesque", "burn", "burning", "burrito", "burro", "burrow", "burst", "bus", "bush", "business", "businessman",
"bust", "bustle", "butane", "butcher", "butler", "butter", "butterfly", "button", "buy", "buyer", "buying", "buzz", "buzzard",
"cabana", "cabbage", "cabin", "cabinet", "cable", "caboose", "cacao", "cactus", "caddy", "cadet", "cafe", "caffeine", "caftan", "cage",
"cake", "calcification", "calculation", "calculator", "calculus", "calendar", "calf", "caliber", "calibre", "calico", "call", "calm",
"calorie", "camel", "cameo", "camera", "camp", "campaign", "campaigning", "campanile", "camper", "campus", "can", "canal", "cancer",
"candelabra", "candidacy", "candidate", "candle", "candy", "cane", "cannibal", "cannon", "canoe", "canon", "canopy", "cantaloupe", "canteen",
"canvas", "cap", "capability", "capacity", "cape", "caper", "capital", "capitalism", "capitulation", "capon", "cappelletti", "cappuccino",
"captain", "caption", "captor", "car", "carabao", "caramel", "caravan", "carbohydrate", "carbon", "carboxyl", "card", "cardboard", "cardigan",
"care", "career", "cargo", "caribou", "carload", "carnation", "carnival", "carol", "carotene", "carp", "carpenter", "carpet", "carpeting",
"carport", "carriage", "carrier", "carrot", "carry", "cart", "cartel", "carter", "cartilage", "cartload", "cartoon", "cartridge", "carving",
"cascade", "case", "casement", "cash", "cashew", "cashier", "casino", "casket", "cassava", "casserole", "cassock", "cast", "castanet",
"castle", "casualty", "cat", "catacomb", "catalogue", "catalysis", "catalyst", "catamaran", "catastrophe", "catch", "catcher", "category",
"caterpillar", "cathedral", "cation", "catsup", "cattle", "cauliflower", "causal", "cause", "causeway", "caution", "cave", "caviar",
"cayenne", "ceiling", "celebration", "celebrity", "celeriac", "celery", "cell", "cellar", "cello", "celsius", "cement", "cemetery", "cenotaph",
"census", "cent", "center", "centimeter", "centre", "centurion", "century", "cephalopod", "ceramic", "ceramics", "cereal", "ceremony",
"certainty", "certificate", "certification", "cesspool", "chafe", "chain", "chainstay", "chair", "chairlift", "chairman", "chairperson",
"chaise", "chalet", "chalice", "chalk", "challenge", "chamber", "champagne", "champion", "championship", "chance", "chandelier", "change",
"channel", "chaos", "chap", "chapel", "chaplain", "chapter", "character", "characteristic", "characterization", "chard", "charge", "charger",
"charity", "charlatan", "charm", "charset", "chart", "charter", "chasm", "chassis", "chastity", "chasuble", "chateau", "chatter", "chauffeur",
"chauvinist", "check", "checkbook", "checking", "checkout", "checkroom", "cheddar", "cheek", "cheer", "cheese", "cheesecake", "cheetah",
"chef", "chem", "chemical", "chemistry", "chemotaxis", "cheque", "cherry", "chess", "chest", "chestnut", "chick", "chicken", "chicory",
"chief", "chiffonier", "child", "childbirth", "childhood", "chili", "chill", "chime", "chimpanzee", "chin", "chinchilla", "chino", "chip",
"chipmunk", "chivalry", "chive", "chives", "chocolate", "choice", "choir", "choker", "cholesterol", "choosing", "chop",
"chops", "chopstick", "chopsticks", "chord", "chorus", "chow", "chowder", "chrome", "chromolithograph", "chronicle", "chronograph", "chronometer",
"chrysalis", "chub", "chuck", "chug", "church", "churn", "chutney", "cicada", "cigarette", "cilantro", "cinder", "cinema", "cinnamon",
"circadian", "circle", "circuit", "circulation", "circumference", "circumstance", "cirrhosis", "cirrus", "citizen", "citizenship", "citron",
"citrus", "city", "civilian", "civilisation", "civilization", "claim", "clam", "clamp", "clan", "clank", "clapboard", "clarification",
"clarinet", "clarity", "clasp", "class", "classic", "classification", "classmate", "classroom", "clause", "clave", "clavicle", "clavier",
"claw", "clay", "cleaner", "clearance", "clearing", "cleat", "cleavage", "clef", "cleft", "clergyman", "cleric", "clerk", "click", "client",
"cliff", "climate", "climb", "clinic", "clip", "clipboard", "clipper", "cloak", "cloakroom", "clock", "clockwork", "clogs", "cloister",
"clone", "close", "closet", "closing", "closure", "cloth", "clothes", "clothing", "cloud", "cloudburst", "clove", "clover", "cloves",
"club", "clue", "cluster", "clutch", "coach", "coal", "coalition", "coast", "coaster", "coat", "cob", "cobbler", "cobweb",
"cock", "cockpit", "cockroach", "cocktail", "cocoa", "coconut", "cod", "code", "codepage", "codling", "codon", "codpiece", "coevolution",
"cofactor", "coffee", "coffin", "cohesion", "cohort", "coil", "coin", "coincidence", "coinsurance", "coke", "cold", "coleslaw", "coliseum",
"collaboration", "collagen", "collapse", "collar", "collard", "collateral", "colleague", "collection", "collectivisation", "collectivization",
"collector", "college", "collision", "colloquy", "colon", "colonial", "colonialism", "colonisation", "colonization", "colony", "color",
"colorlessness", "colt", "column", "columnist", "comb", "combat", "combination", "combine", "comeback", "comedy", "comestible", "comfort",
"comfortable", "comic", "comics", "comma", "command", "commander", "commandment", "comment", "commerce", "commercial", "commission",
"commitment", "committee", "commodity", "common", "commonsense", "commotion", "communicant", "communication", "communion", "communist",
"community", "commuter", "company", "comparison", "compass", "compassion", "compassionate", "compensation", "competence", "competition",
"competitor", "complaint", "complement", "completion", "complex", "complexity", "compliance", "complication", "complicity", "compliment",
"component", "comportment", "composer", "composite", "composition", "compost", "comprehension", "compress", "compromise", "comptroller",
"compulsion", "computer", "comradeship", "con", "concentrate", "concentration", "concept", "conception", "concern", "concert", "conclusion",
"concrete", "condition", "conditioner", "condominium", "condor", "conduct", "conductor", "cone", "confectionery", "conference", "confidence",
"confidentiality", "configuration", "confirmation", "conflict", "conformation", "confusion", "conga", "congo", "congregation", "congress",
"congressman", "congressperson", "conifer", "connection", "connotation", "conscience", "consciousness", "consensus", "consent", "consequence",
"conservation", "conservative", "consideration", "consignment", "consist", "consistency", "console", "consonant", "conspiracy", "conspirator",
"constant", "constellation", "constitution", "constraint", "construction", "consul", "consulate", "consulting", "consumer", "consumption",
"contact", "contact lens", "contagion", "container", "content", "contention", "contest", "context", "continent", "contingency", "continuity",
"contour", "contract", "contractor", "contrail", "contrary", "contrast", "contribution", "contributor", "control", "controller", "controversy",
"convection", "convenience", "convention", "conversation", "conversion", "convert", "convertible", "conviction", "cook", "cookbook",
"cookie", "cooking", "coonskin", "cooperation", "coordination", "coordinator", "cop", "cope", "copper", "copy", "copying",
"copyright", "copywriter", "coral", "cord", "corduroy", "core", "cork", "cormorant", "corn", "corner", "cornerstone", "cornet", "cornflakes",
"cornmeal", "corporal", "corporation", "corporatism", "corps", "corral", "correspondence", "correspondent", "corridor", "corruption",
"corsage", "cosset", "cost", "costume", "cot", "cottage", "cotton", "couch", "cougar", "cough", "council", "councilman", "councilor",
"councilperson", "counsel", "counseling", "counselling", "counsellor", "counselor", "count", "counter", "counterpart",
"counterterrorism", "countess", "country", "countryside", "county", "couple", "coupon", "courage", "course", "court", "courthouse", "courtroom",
"cousin", "covariate", "cover", "coverage", "coverall", "cow", "cowbell", "cowboy", "coyote", "crab", "crack", "cracker", "crackers",
"cradle", "craft", "craftsman", "cranberry", "crane", "cranky", "crash", "crate", "cravat", "craw", "crawdad", "crayfish", "crayon",
"crazy", "cream", "creation", "creationism", "creationist", "creative", "creativity", "creator", "creature", "creche", "credential",
"credenza", "credibility", "credit", "creditor", "creek", "creme brulee", "crepe", "crest", "crew", "crewman", "crewmate", "crewmember",
"crewmen", "cria", "crib", "cribbage", "cricket", "cricketer", "crime", "criminal", "crinoline", "crisis", "crisp", "criteria", "criterion",
"critic", "criticism", "crocodile", "crocus", "croissant", "crook", "crop", "cross", "crotch",
"croup", "crow", "crowd", "crown", "crucifixion", "crude", "cruelty", "cruise", "crumb", "crunch", "crusader", "crush", "crust", "cry",
"crystal", "crystallography", "cub", "cube", "cuckoo", "cucumber", "cue", "cuisine", "cultivar", "cultivator", "culture",
"culvert", "cummerbund", "cup", "cupboard", "cupcake", "cupola", "curd", "cure", "curio", "curiosity", "curl", "curler", "currant", "currency",
"current", "curriculum", "curry", "curse", "cursor", "curtailment", "curtain", "curve", "cushion", "custard", "custody", "custom", "customer",
"cut", "cuticle", "cutlet", "cutover", "cutting", "cyclamen", "cycle", "cyclone", "cyclooxygenase", "cygnet", "cylinder", "cymbal", "cynic",
"cyst", "cytokine", "cytoplasm", "dad", "daddy", "daffodil", "dagger", "dahlia", "daikon", "daily", "dairy", "daisy", "dam", "damage",
"dame", "dance", "dancer", "dancing", "dandelion", "danger", "dare", "dark", "darkness", "darn", "dart", "dash", "dashboard",
"data", "database", "date", "daughter", "dawn", "day", "daybed", "daylight", "dead", "deadline", "deal", "dealer", "dealing", "dearest",
"death", "deathwatch", "debate", "debris", "debt", "debtor", "decade", "decadence", "decency", "decimal", "decision",
"deck", "declaration", "declination", "decline", "decoder", "decongestant", "decoration", "decrease", "decryption", "dedication", "deduce",
"deduction", "deed", "deep", "deer", "default", "defeat", "defendant", "defender", "defense", "deficit", "definition", "deformation",
"degradation", "degree", "delay", "deliberation", "delight", "delivery", "demand", "democracy", "democrat", "demon", "demur", "den",
"denim", "denominator", "density", "dentist", "deodorant", "department", "departure", "dependency", "dependent", "deployment", "deposit",
"deposition", "depot", "depression", "depressive", "depth", "deputy", "derby", "derivation", "derivative", "derrick", "descendant", "descent",
"description", "desert", "design", "designation", "designer", "desire", "desk", "desktop", "dessert", "destination", "destiny", "destroyer",
"destruction", "detail", "detainee", "detainment", "detection", "detective", "detector", "detention", "determination", "detour", "devastation",
"developer", "developing", "development", "developmental", "deviance", "deviation", "device", "devil", "dew", "dhow", "diabetes", "diadem",
"diagnosis", "diagram", "dial", "dialect", "dialogue", "diam", "diamond", "diaper", "diaphragm", "diarist", "diary", "dibble", "dickey", "dictaphone", "dictator", "diction", "dictionary", "die", "diesel", "diet", "difference", "differential", "difficulty", "diffuse",
"dig", "digestion", "digestive", "digger", "digging", "digit", "dignity", "dilapidation", "dill", "dilution", "dime", "dimension", "dimple",
"diner", "dinghy", "dining", "dinner", "dinosaur", "dioxide", "dip", "diploma", "diplomacy", "dipstick", "direction", "directive", "director",
"directory", "dirndl", "dirt", "disability", "disadvantage", "disagreement", "disappointment", "disarmament", "disaster", "discharge",
"discipline", "disclaimer", "disclosure", "disco", "disconnection", "discount", "discourse", "discovery", "discrepancy", "discretion",
"discrimination", "discussion", "disdain", "disease", "disembodiment", "disengagement", "disguise", "disgust", "dish", "dishwasher",
"disk", "disparity", "dispatch", "displacement", "display", "disposal", "disposer", "disposition", "dispute", "disregard", "disruption",
"dissemination", "dissonance", "distance", "distinction", "distortion", "distribution", "distributor", "district", "divalent", "divan",
"diver", "diversity", "divide", "dividend", "divider", "divine", "diving", "division", "divorce", "doc", "dock", "doctor", "doctorate",
"doctrine", "document", "documentary", "documentation", "doe", "dog", "doggie", "dogsled", "dogwood", "doing", "doll", "dollar", "dollop",
"dolman", "dolor", "dolphin", "domain", "dome", "domination", "donation", "donkey", "donor", "donut", "door", "doorbell", "doorknob",
"doorpost", "doorway", "dory", "dose", "dot", "double", "doubling", "doubt", "doubter", "dough", "doughnut", "down", "downfall", "downforce",
"downgrade", "download", "downstairs", "downtown", "downturn", "dozen", "draft", "drag", "dragon", "dragonfly", "dragonfruit", "dragster",
"drain", "drainage", "drake", "drama", "dramaturge", "drapes", "draw", "drawbridge", "drawer", "drawing", "dream", "dreamer", "dredger",
"dress", "dresser", "dressing", "drill", "drink", "drinking", "drive", "driver", "driveway", "driving", "drizzle", "dromedary", "drop",
"drudgery", "drug", "drum", "drummer", "drunk", "dryer", "duck", "duckling", "dud", "dude", "due", "duel", "dueling", "duffel", "dugout",
"dulcimer", "dumbwaiter", "dump", "dump truck", "dune", "dune buggy", "dungarees", "dungeon", "duplexer", "duration", "durian", "dusk",
"dust", "dust storm", "duster", "duty", "dwarf", "dwell", "dwelling", "dynamics", "dynamite", "dynamo", "dynasty", "dysfunction",
"eagle", "eaglet", "ear", "eardrum", "earmuffs", "earnings", "earplug", "earring", "earrings", "earth", "earthquake",
"earthworm", "ease", "easel", "east", "eating", "eaves", "eavesdropper", "ecclesia", "echidna", "eclipse", "ecliptic", "ecology", "economics",
"economy", "ecosystem", "ectoderm", "ectodermal", "ecumenist", "eddy", "edge", "edger", "edible", "editing", "edition", "editor", "editorial",
"education", "eel", "effacement", "effect", "effective", "effectiveness", "effector", "efficacy", "efficiency", "effort", "egg", "egghead",
"eggnog", "eggplant", "ego", "eicosanoid", "ejector", "elbow", "elderberry", "election", "electricity", "electrocardiogram", "electronics",
"element", "elephant", "elevation", "elevator", "eleventh", "elf", "elicit", "eligibility", "elimination", "elite", "elixir", "elk",
"ellipse", "elm", "elongation", "elver", "email", "emanate", "embarrassment", "embassy", "embellishment", "embossing", "embryo", "emerald",
"emergence", "emergency", "emergent", "emery", "emission", "emitter", "emotion", "emphasis", "empire", "employ", "employee", "employer",
"employment", "empowerment", "emu", "enactment", "encirclement", "enclave", "enclosure", "encounter", "encouragement", "encyclopedia",
"end", "endive", "endoderm", "endorsement", "endothelium", "endpoint", "enemy", "energy", "enforcement", "engagement", "engine", "engineer",
"engineering", "enigma", "enjoyment", "enquiry", "enrollment", "enterprise", "entertainment", "enthusiasm", "entirety", "entity", "entrance",
"entree", "entrepreneur", "entry", "envelope", "environment", "envy", "enzyme", "epauliere", "epee", "ephemera", "ephemeris", "ephyra",
"epic", "episode", "epithelium", "epoch", "eponym", "epoxy", "equal", "equality", "equation", "equinox", "equipment", "equity", "equivalent",
"era", "eraser", "erection", "erosion", "error", "escalator", "escape", "escort", "espadrille", "espalier", "essay", "essence", "essential",
"establishment", "estate", "estimate", "estrogen", "estuary", "eternity", "ethernet", "ethics", "ethnicity", "ethyl", "euphonium", "eurocentrism",
"evaluation", "evaluator", "evaporation", "eve", "evening", "event", "everybody", "everyone", "everything", "eviction",
"evidence", "evil", "evocation", "evolution", "exaggeration", "exam", "examination", "examiner", "example",
"exasperation", "excellence", "exception", "excerpt", "excess", "exchange", "excitement", "exclamation", "excursion", "excuse", "execution",
"executive", "executor", "exercise", "exhaust", "exhaustion", "exhibit", "exhibition", "exile", "existence", "exit", "exocrine", "expansion",
"expansionism", "expectancy", "expectation", "expedition", "expense", "experience", "experiment", "experimentation", "expert", "expertise",
"explanation", "exploration", "explorer", "explosion", "export", "expose", "exposition", "exposure", "expression", "extension", "extent",
"exterior", "external", "extinction", "extreme", "extremist", "eye", "eyeball", "eyebrow", "eyebrows", "eyeglasses", "eyelash", "eyelashes",
"eyelid", "eyelids", "eyeliner", "eyestrain", "eyrie", "fabric", "face", "facelift", "facet", "facility", "facsimile", "fact", "factor",
"factory", "faculty", "fahrenheit", "fail", "failure", "fairness", "fairy", "faith", "faithful", "fall", "fallacy", "fame",
"familiar", "familiarity", "family", "fan", "fang", "fanlight", "fanny", "fantasy", "farm", "farmer", "farming", "farmland",
"farrow", "fascia", "fashion", "fat", "fate", "father", "fatigue", "fatigues", "faucet", "fault", "fav", "fava", "favor",
"favorite", "fawn", "fax", "fear", "feast", "feather", "feature", "fedelini", "federation", "fedora", "fee", "feed", "feedback", "feeding",
"feel", "feeling", "fellow", "felony", "female", "fen", "fence", "fencing", "fender", "feng", "fennel", "ferret", "ferry", "ferryboat",
"fertilizer", "festival", "fetus", "few", "fiber", "fiberglass", "fibre", "fibroblast", "fibrosis", "ficlet", "fiction", "fiddle", "field",
"fiery", "fiesta", "fifth", "fig", "fight", "fighter", "figure", "figurine", "file", "filing", "fill", "fillet", "filly", "film", "filter",
"filth", "final", "finance", "financing", "finding", "fine", "finer", "finger", "fingerling", "fingernail", "finish", "finisher", "fir",
"fire", "fireman", "fireplace", "firewall", "firm", "first", "fish", "fishbone", "fisherman", "fishery", "fishing", "fishmonger", "fishnet",
"fisting", "fit", "fitness", "fix", "fixture", "flag", "flair", "flame", "flan", "flanker", "flare", "flash", "flat", "flatboat", "flavor",
"flax", "fleck", "fledgling", "fleece", "flesh", "flexibility", "flick", "flicker", "flight", "flint", "flintlock", "flock",
"flood", "floodplain", "floor", "floozie", "flour", "flow", "flower", "flu", "flugelhorn", "fluke", "flume", "flung", "flute", "fly",
"flytrap", "foal", "foam", "fob", "focus", "fog", "fold", "folder", "folk", "folklore", "follower", "following", "fondue", "font", "food",
"foodstuffs", "fool", "foot", "footage", "football", "footnote", "footprint", "footrest", "footstep", "footstool", "footwear", "forage",
"forager", "foray", "force", "ford", "forearm", "forebear", "forecast", "forehead", "foreigner", "forelimb", "forest", "forestry", "forever",
"forgery", "fork", "form", "formal", "formamide", "format", "formation", "former", "formicarium", "formula", "fort", "forte", "fortnight",
"fortress", "fortune", "forum", "foundation", "founder", "founding", "fountain", "fourths", "fowl", "fox", "foxglove", "fraction", "fragrance",
"frame", "framework", "fratricide", "fraud", "fraudster", "freak", "freckle", "freedom", "freelance", "freezer", "freezing", "freight",
"freighter", "frenzy", "freon", "frequency", "fresco", "friction", "fridge", "friend", "friendship", "fries", "frigate", "fright", "fringe",
"fritter", "frock", "frog", "front", "frontier", "frost", "frosting", "frown", "fruit", "frustration", "fry", "fuel", "fugato",
"fulfillment", "full", "fun", "function", "functionality", "fund", "funding", "fundraising", "funeral", "fur", "furnace", "furniture",
"furry", "fusarium", "futon", "future", "gadget", "gaffe", "gaffer", "gain", "gaiters", "gale", "gallery", "galley",
"gallon", "galoshes", "gambling", "game", "gamebird", "gaming", "gander", "gang", "gap", "garage", "garb", "garbage", "garden",
"garlic", "garment", "garter", "gas", "gasket", "gasoline", "gasp", "gastronomy", "gastropod", "gate", "gateway", "gather", "gathering",
"gator", "gauge", "gauntlet", "gavel", "gazebo", "gazelle", "gear", "gearshift", "geek", "gel", "gelatin", "gelding", "gem", "gemsbok",
"gender", "gene", "general", "generation", "generator", "generosity", "genetics", "genie", "genius", "genocide", "genre", "gentleman",
"geography", "geology", "geometry", "geranium", "gerbil", "gesture", "geyser", "gherkin", "ghost", "giant", "gift", "gig", "gigantism",
"giggle", "ginger", "gingerbread", "ginseng", "giraffe", "girdle", "girl", "girlfriend", "git", "glacier", "gladiolus", "glance", "gland",
"glass", "glasses", "glee", "glen", "glider", "gliding", "glimpse", "globe", "glockenspiel", "gloom", "glory", "glove", "glow", "glucose",
"glue", "glut", "glutamate", "gnat", "gnu", "goal", "goat", "gobbler", "god", "goddess", "godfather", "godmother", "godparent",
"goggles", "going", "gold", "goldfish", "golf", "gondola", "gong", "good", "goodbye", "goodie", "goodness", "goodnight",
"goodwill", "goose", "gopher", "gorilla", "gosling", "gossip", "governance", "government", "governor", "gown", "grace", "grade",
"gradient", "graduate", "graduation", "graffiti", "graft", "grain", "gram", "grammar", "gran", "grand", "grandchild", "granddaughter",
"grandfather", "grandma", "grandmom", "grandmother", "grandpa", "grandparent", "grandson", "granny", "granola", "grant", "grape", "grapefruit",
"graph", "graphic", "grasp", "grass", "grasshopper", "grassland", "gratitude", "gravel", "gravitas", "gravity", "gravy", "gray", "grease",
"greatness", "greed", "green", "greenhouse", "greens", "grenade", "grey", "grid", "grief",
"grill", "grin", "grip", "gripper", "grit", "grocery", "ground", "group", "grouper", "grouse", "grove", "growth", "grub", "guacamole",
"guarantee", "guard", "guava", "guerrilla", "guess", "guest", "guestbook", "guidance", "guide", "guideline", "guilder", "guilt", "guilty",
"guinea", "guitar", "guitarist", "gum", "gumshoe", "gun", "gunpowder", "gutter", "guy", "gym", "gymnast", "gymnastics", "gynaecology",
"gyro", "habit", "habitat", "hacienda", "hacksaw", "hackwork", "hail", "hair", "haircut", "hake", "half",
"halibut", "hall", "halloween", "hallway", "halt", "ham", "hamburger", "hammer", "hammock", "hamster", "hand", "handball",
"handful", "handgun", "handicap", "handle", "handlebar", "handmaiden", "handover", "handrail", "handsaw", "hanger", "happening", "happiness",
"harald", "harbor", "harbour", "hardboard", "hardcover", "hardening", "hardhat", "hardship", "hardware", "hare", "harm",
"harmonica", "harmonise", "harmonize", "harmony", "harp", "harpooner", "harpsichord", "harvest", "harvester", "hash", "hashtag", "hassock",
"haste", "hat", "hatbox", "hatchet", "hatchling", "hate", "hatred", "haunt", "haven", "haversack", "havoc", "hawk", "hay", "haze", "hazel",
"hazelnut", "head", "headache", "headlight", "headline", "headphones", "headquarters", "headrest", "health", "hearing",
"hearsay", "heart", "heartache", "heartbeat", "hearth", "hearthside", "heartwood", "heat", "heater", "heating", "heaven",
"heavy", "hectare", "hedge", "hedgehog", "heel", "heifer", "height", "heir", "heirloom", "helicopter", "helium", "hell", "hellcat", "hello",
"helmet", "helo", "help", "hemisphere", "hemp", "hen", "hepatitis", "herb", "herbs", "heritage", "hermit", "hero", "heroine", "heron",
"herring", "hesitation", "hexagon", "heyday", "hiccups", "hide", "hierarchy", "high", "highland", "highlight",
"highway", "hike", "hiking", "hill", "hint", "hip", "hippodrome", "hippopotamus", "hire", "hiring", "historian", "history", "hit", "hive",
"hobbit", "hobby", "hockey", "hoe", "hog", "hold", "holder", "hole", "holiday", "home", "homeland", "homeownership", "hometown", "homework",
"homicide", "homogenate", "homonym", "honesty", "honey", "honeybee", "honeydew", "honor", "honoree", "hood",
"hoof", "hook", "hop", "hope", "hops", "horde", "horizon", "hormone", "horn", "hornet", "horror", "horse", "horseradish", "horst", "hose",
"hosiery", "hospice", "hospital", "hospitalisation", "hospitality", "hospitalization", "host", "hostel", "hostess", "hotdog", "hotel",
"hound", "hour", "hourglass", "house", "houseboat", "household", "housewife", "housework", "housing", "hovel", "hovercraft", "howard",
"howitzer", "hub", "hubcap", "hubris", "hug", "hugger", "hull", "human", "humanity", "humidity", "hummus", "humor", "humour", "hunchback",
"hundred", "hunger", "hunt", "hunter", "hunting", "hurdle", "hurdler", "hurricane", "hurry", "hurt", "husband", "hut", "hutch", "hyacinth",
"hybridisation", "hybridization", "hydrant", "hydraulics", "hydrocarb", "hydrocarbon", "hydrofoil", "hydrogen", "hydrolyse", "hydrolysis",
"hydrolyze", "hydroxyl", "hyena", "hygienic", "hype", "hyphenation", "hypochondria", "hypothermia", "hypothesis", "ice",
"iceberg", "icebreaker", "icecream", "icicle", "icing", "icon", "icy", "id", "idea", "ideal", "identification", "identity", "ideology",
"idiom", "idiot", "igloo", "ignorance", "ignorant", "ikebana", "illegal", "illiteracy", "illness", "illusion", "illustration", "image",
"imagination", "imbalance", "imitation", "immigrant", "immigration", "immortal", "impact", "impairment", "impala", "impediment", "implement",
"implementation", "implication", "import", "importance", "impostor", "impress", "impression", "imprisonment", "impropriety", "improvement",
"impudence", "impulse", "inability", "inauguration", "inbox", "incandescence", "incarnation", "incense", "incentive",
"inch", "incidence", "incident", "incision", "inclusion", "income", "incompetence", "inconvenience", "increase", "incubation", "independence",
"independent", "index", "indication", "indicator", "indigence", "individual", "industrialisation", "industrialization", "industry", "inequality",
"inevitable", "infancy", "infant", "infarction", "infection", "infiltration", "infinite", "infix", "inflammation", "inflation", "influence",
"influx", "info", "information", "infrastructure", "infusion", "inglenook", "ingrate", "ingredient", "inhabitant", "inheritance", "inhibition",
"inhibitor", "initial", "initialise", "initialize", "initiative", "injunction", "injury", "injustice", "ink", "inlay", "inn", "innervation",
"innocence", "innocent", "innovation", "input", "inquiry", "inscription", "insect", "insectarium", "insert", "inside", "insight", "insolence",
"insomnia", "inspection", "inspector", "inspiration", "installation", "instance", "instant", "instinct", "institute", "institution",
"instruction", "instructor", "instrument", "instrumentalist", "instrumentation", "insulation", "insurance", "insurgence", "insurrection",
"integer", "integral", "integration", "integrity", "intellect", "intelligence", "intensity", "intent", "intention", "intentionality",
"interaction", "interchange", "interconnection", "intercourse", "interest", "interface", "interferometer", "interior", "interject", "interloper",
"internet", "interpretation", "interpreter", "interval", "intervenor", "intervention", "interview", "interviewer", "intestine", "introduction",
"intuition", "invader", "invasion", "invention", "inventor", "inventory", "inverse", "inversion", "investigation", "investigator", "investment",
"investor", "invitation", "invite", "invoice", "involvement", "iridescence", "iris", "iron", "ironclad", "irony", "irrigation", "ischemia",
"island", "isogloss", "isolation", "issue", "item", "itinerary", "ivory", "jack", "jackal", "jacket", "jackfruit", "jade", "jaguar",
"jail", "jailhouse", "jalapeño", "jam", "jar", "jasmine", "jaw", "jazz", "jealousy", "jeans", "jeep", "jelly", "jellybeans", "jellyfish",
"jerk", "jet", "jewel", "jeweller", "jewellery", "jewelry", "jicama", "jiffy", "job", "jockey", "jodhpurs", "joey", "jogging", "joint",
"joke", "jot", "journal", "journalism", "journalist", "journey", "joy", "judge", "judgment", "judo", "jug", "juggernaut", "juice", "julienne",
"jumbo", "jump", "jumper", "jumpsuit", "jungle", "junior", "junk", "junker", "junket", "jury", "justice", "justification", "jute", "kale",
"kamikaze", "kangaroo", "karate", "kayak", "kazoo", "kebab", "keep", "keeper", "kendo", "kennel", "ketch", "ketchup", "kettle", "kettledrum",
"key", "keyboard", "keyboarding", "keystone", "kick", "kid", "kidney", "kielbasa", "kill", "killer", "killing", "kilogram",
"kilometer", "kilt", "kimono", "kinase", "kind", "kindness", "king", "kingdom", "kingfish", "kiosk", "kiss", "kit", "kitchen", "kite",
"kitsch", "kitten", "kitty", "kiwi", "knee", "kneejerk", "knickers", "knife", "knight", "knitting", "knock", "knot",
"knowledge", "knuckle", "koala", "kohlrabi", "kumquat", "lab", "label", "labor", "laboratory", "laborer", "labour", "labourer", "lace",
"lack", "lacquerware", "lad", "ladder", "ladle", "lady", "ladybug", "lag", "lake", "lamb", "lambkin", "lament", "lamp", "lanai", "land",
"landform", "landing", "landmine", "landscape", "lane", "language", "lantern", "lap", "laparoscope", "lapdog", "laptop", "larch", "lard",
"larder", "lark", "larva", "laryngitis", "lasagna", "lashes", "last", "latency", "latex", "lathe", "latitude", "latte", "latter", "laugh",
"laughter", "laundry", "lava", "law", "lawmaker", "lawn", "lawsuit", "lawyer", "lay", "layer", "layout", "lead", "leader", "leadership",
"leading", "leaf", "league", "leaker", "leap", "learning", "leash", "leather", "leave", "leaver", "lecture", "leek", "leeway", "left",
"leg", "legacy", "legal", "legend", "legging", "legislation", "legislator", "legislature", "legitimacy", "legume", "leisure", "lemon",
"lemonade", "lemur", "lender", "lending", "length", "lens", "lentil", "leopard", "leprosy", "leptocephalus", "lesson", "letter",
"lettuce", "level", "lever", "leverage", "leveret", "liability", "liar", "liberty", "libido", "library", "licence", "license", "licensing",
"licorice", "lid", "lie", "lieu", "lieutenant", "life", "lifestyle", "lifetime", "lift", "ligand", "light", "lighting", "lightning",
"lightscreen", "ligula", "likelihood", "likeness", "lilac", "lily", "limb", "lime", "limestone", "limit", "limitation", "limo", "line",
"linen", "liner", "linguist", "linguistics", "lining", "link", "linkage", "linseed", "lion", "lip", "lipid", "lipoprotein", "lipstick",
"liquid", "liquidity", "liquor", "list", "listening", "listing", "literate", "literature", "litigation", "litmus", "litter", "littleneck",
"liver", "livestock", "living", "lizard", "llama", "load", "loading", "loaf", "loafer", "loan", "lobby", "lobotomy", "lobster", "local",
"locality", "location", "lock", "locker", "locket", "locomotive", "locust", "lode", "loft", "log", "loggia", "logic", "login", "logistics",
"logo", "loincloth", "lollipop", "loneliness", "longboat", "longitude", "look", "lookout", "loop", "loophole", "loquat", "lord", "loss",
"lot", "lotion", "lottery", "lounge", "louse", "lout", "love", "lover", "lox", "loyalty", "luck", "luggage", "lumber", "lumberman", "lunch",
"luncheonette", "lunchmeat", "lunchroom", "lung", "lunge", "lust", "lute", "luxury", "lychee", "lycra", "lye", "lymphocyte", "lynx",
"lyocell", "lyre", "lyrics", "lysine", "mRNA", "macadamia", "macaroni", "macaroon", "macaw", "machine", "machinery", "macrame", "macro",
"macrofauna", "madam", "maelstrom", "maestro", "magazine", "maggot", "magic", "magnet", "magnitude", "maid", "maiden", "mail", "mailbox",
"mailer", "mailing", "mailman", "main", "mainland", "mainstream", "maintainer", "maintenance", "maize", "major", "majority",
"makeover", "maker", "makeup", "making", "male", "malice", "mall", "mallard", "mallet", "malnutrition", "mama", "mambo", "mammoth", "man",
"manacle", "management", "manager", "manatee", "mandarin", "mandate", "mandolin", "mangle", "mango", "mangrove", "manhunt", "maniac",
"manicure", "manifestation", "manipulation", "mankind", "manner", "manor", "mansard", "manservant", "mansion", "mantel", "mantle", "mantua",
"manufacturer", "manufacturing", "many", "map", "maple", "mapping", "maracas", "marathon", "marble", "march", "mare", "margarine", "margin",
"mariachi", "marimba", "marines", "marionberry", "mark", "marker", "market", "marketer", "marketing", "marketplace", "marksman", "markup",
"marmalade", "marriage", "marsh", "marshland", "marshmallow", "marten", "marxism", "mascara", "mask", "masonry", "mass", "massage", "mast",
"master", "masterpiece", "mastication", "mastoid", "mat", "match", "matchmaker", "mate", "material", "maternity", "math", "mathematics",
"matrix", "matter", "mattock", "mattress", "max", "maximum", "maybe", "mayonnaise", "mayor", "meadow", "meal", "mean", "meander", "meaning",
"means", "meantime", "measles", "measure", "measurement", "meat", "meatball", "meatloaf", "mecca", "mechanic", "mechanism", "med", "medal",
"media", "median", "medication", "medicine", "medium", "meet", "meeting", "melatonin", "melody", "melon", "member", "membership", "membrane",
"meme", "memo", "memorial", "memory", "men", "menopause", "menorah", "mention", "mentor", "menu", "merchandise", "merchant", "mercury",
"meridian", "meringue", "merit", "mesenchyme", "mess", "message", "messenger", "messy", "metabolite", "metal", "metallurgist", "metaphor",
"meteor", "meteorology", "meter", "methane", "method", "methodology", "metric", "metro", "metronome", "mezzanine", "microlending", "micronutrient",
"microphone", "microwave", "midden", "middle", "middleman", "midline", "midnight", "midwife", "might", "migrant", "migration",
"mile", "mileage", "milepost", "milestone", "military", "milk", "milkshake", "mill", "millennium", "millet", "millimeter", "million",
"millisecond", "millstone", "mime", "mimosa", "min", "mincemeat", "mind", "mine", "mineral", "mineshaft", "mini", "minibus",
"minimalism", "minimum", "mining", "minion", "minister", "mink", "minnow", "minor", "minority", "mint", "minute", "miracle",
"mirror", "miscarriage", "miscommunication", "misfit", "misnomer", "misogyny", "misplacement", "misreading", "misrepresentation", "miss",
"missile", "mission", "missionary", "mist", "mistake", "mister", "misunderstand", "miter", "mitten", "mix", "mixer", "mixture", "moai",
"moat", "mob", "mobile", "mobility", "mobster", "moccasins", "mocha", "mochi", "mode", "model", "modeling", "modem", "modernist", "modernity",
"modification", "molar", "molasses", "molding", "mole", "molecule", "mom", "moment", "monastery", "monasticism", "money", "monger", "monitor",
"monitoring", "monk", "monkey", "monocle", "monopoly", "monotheism", "monsoon", "monster", "month", "monument", "mood", "moody", "moon",
"moonlight", "moonscape", "moonshine", "moose", "mop", "morale", "morbid", "morbidity", "morning", "moron", "morphology", "morsel", "mortal",
"mortality", "mortgage", "mortise", "mosque", "mosquito", "most", "motel", "moth", "mother", "motion", "motivation",
"motive", "motor", "motorboat", "motorcar", "motorcycle", "mound", "mountain", "mouse", "mouser", "mousse", "moustache", "mouth", "mouton",
"movement", "mover", "movie", "mower", "mozzarella", "mud", "muffin", "mug", "mukluk", "mule", "multimedia", "murder", "muscat", "muscatel",
"muscle", "musculature", "museum", "mushroom", "music", "musician", "muskrat", "mussel", "mustache", "mustard",
"mutation", "mutt", "mutton", "mycoplasma", "mystery", "myth", "mythology", "nail", "name", "naming", "nanoparticle", "napkin", "narrative",
"nasal", "nation", "nationality", "native", "naturalisation", "nature", "navigation", "necessity", "neck", "necklace", "necktie", "nectar",
"nectarine", "need", "needle", "neglect", "negligee", "negotiation", "neighbor", "neighborhood", "neighbour", "neighbourhood", "neologism",
"neon", "neonate", "nephew", "nerve", "nest", "nestling", "nestmate", "net", "netball", "netbook", "netsuke", "network", "networking",
"neurobiologist", "neuron", "neuropathologist", "neuropsychiatry", "news", "newsletter", "newspaper", "newsprint", "newsstand", "nexus",
"nibble", "nicety", "niche", "nick", "nickel", "nickname", "niece", "night", "nightclub", "nightgown", "nightingale", "nightlife", "nightlight",
"nightmare", "ninja", "nit", "nitrogen", "nobody", "nod", "node", "noir", "noise", "nonbeliever", "nonconformist", "nondisclosure", "nonsense",
"noodle", "noodles", "noon", "norm", "normal", "normalisation", "normalization", "north", "nose", "notation", "note", "notebook", "notepad",
"nothing", "notice", "notion", "notoriety", "nougat", "noun", "nourishment", "novel", "nucleotidase", "nucleotide", "nudge", "nuke",
"number", "numeracy", "numeric", "numismatist", "nun", "nurse", "nursery", "nursing", "nurture", "nut", "nutmeg", "nutrient", "nutrition",
"nylon", "nymph", "oak", "oar", "oasis", "oat", "oatmeal", "oats", "obedience", "obesity", "obi", "object", "objection", "objective",
"obligation", "oboe", "observation", "observatory", "obsession", "obsidian", "obstacle", "occasion", "occupation", "occurrence", "ocean",
"ocelot", "octagon", "octave", "octavo", "octet", "octopus", "odometer", "odyssey", "oeuvre", "offence", "offense", "offer",
"offering", "office", "officer", "official", "offset", "oil", "okra", "oldie", "oleo", "olive", "omega", "omelet", "omission", "omnivore",
"oncology", "onion", "online", "onset", "opening", "opera", "operating", "operation", "operator", "ophthalmologist", "opinion", "opium",
"opossum", "opponent", "opportunist", "opportunity", "opposite", "opposition", "optimal", "optimisation", "optimist", "optimization",
"option", "orange", "orangutan", "orator", "orchard", "orchestra", "orchid", "order", "ordinary", "ordination", "ore", "oregano", "organ",
"organisation", "organising", "organization", "organizing", "orient", "orientation", "origin", "original", "originality", "ornament",
"osmosis", "osprey", "ostrich", "other", "otter", "ottoman", "ounce", "outback", "outcome", "outfielder", "outfit", "outhouse", "outlaw",
"outlay", "outlet", "outline", "outlook", "output", "outrage", "outrigger", "outrun", "outset", "outside", "oval", "ovary", "oven", "overcharge",
"overclocking", "overcoat", "overexertion", "overflight", "overhead", "overheard", "overload", "overnighter", "overshoot", "oversight",
"overview", "overweight", "owl", "owner", "ownership", "ox", "oxford", "oxygen", "oyster", "ozone", "pace", "pacemaker", "pack", "package",
"packaging", "packet", "pad", "paddle", "paddock", "pagan", "page", "pagoda", "pail", "pain", "paint", "painter", "painting", "paintwork",
"pair", "pajamas", "palace", "palate", "palm", "pamphlet", "pan", "pancake", "pancreas", "panda", "panel", "panic", "pannier", "panpipe",
"pansy", "panther", "panties", "pantologist", "pantology", "pantry", "pants", "pantsuit", "panty", "pantyhose", "papa", "papaya", "paper",
"paperback", "paperwork", "parable", "parachute", "parade", "paradise", "paragraph", "parallelogram", "paramecium", "paramedic", "parameter",
"paranoia", "parcel", "parchment", "pard", "pardon", "parent", "parenthesis", "parenting", "park", "parka", "parking", "parliament",
"parole", "parrot", "parser", "parsley", "parsnip", "part", "participant", "participation", "particle", "particular", "partner", "partnership",
"partridge", "party", "pass", "passage", "passbook", "passenger", "passing", "passion", "passive", "passport", "password", "past", "pasta",
"paste", "pastor", "pastoralist", "pastry", "pasture", "pat", "patch", "pate", "patent", "patentee", "path", "pathogenesis", "pathology",
"pathway", "patience", "patient", "patina", "patio", "patriarch", "patrimony", "patriot", "patrol", "patroller", "patrolling", "patron",
"pattern", "patty", "pattypan", "pause", "pavement", "pavilion", "paw", "pawnshop", "pay", "payee", "payment", "payoff", "pea", "peace",
"peach", "peacoat", "peacock", "peak", "peanut", "pear", "pearl", "peasant", "pecan", "pecker", "pedal", "peek", "peen", "peer",
"pegboard", "pelican", "pelt", "pen", "penalty", "pence", "pencil", "pendant", "pendulum", "penguin", "penicillin", "peninsula", "penis",
"pennant", "penny", "pension", "pentagon", "peony", "people", "pepper", "pepperoni", "percent", "percentage", "perception", "perch",
"perennial", "perfection", "performance", "perfume", "period", "periodical", "peripheral", "permafrost", "permission", "permit", "perp",
"perpendicular", "persimmon", "person", "personal", "personality", "personnel", "perspective", "pest", "pet", "petal", "petition", "petitioner",
"petticoat", "pew", "pharmacist", "pharmacopoeia", "phase", "pheasant", "phenomenon", "phenotype", "pheromone", "philanthropy", "philosopher",
"philosophy", "phone", "phosphate", "photo", "photodiode", "photograph", "photographer", "photography", "photoreceptor", "phrase", "phrasing",
"physical", "physics", "physiology", "pianist", "piano", "piccolo", "pick", "pickax", "pickaxe", "picket", "pickle", "pickup", "picnic",
"picture", "picturesque", "pie", "piece", "pier", "piety", "pig", "pigeon", "piglet", "pigpen", "pigsty", "pike", "pilaf", "pile", "pilgrim",
"pilgrimage", "pill", "pillar", "pillbox", "pillow", "pilot", "pimp", "pimple", "pin", "pinafore", "pine", "pineapple",
"pinecone", "ping", "pink", "pinkie", "pinot", "pinstripe", "pint", "pinto", "pinworm", "pioneer", "pipe", "pipeline", "piracy", "pirate",
"pistol", "pit", "pita", "pitch", "pitcher", "pitching", "pith", "pizza", "place", "placebo", "placement", "placode", "plagiarism",
"plain", "plaintiff", "plan", "plane", "planet", "planning", "plant", "plantation", "planter", "planula", "plaster", "plasterboard",
"plastic", "plate", "platelet", "platform", "platinum", "platter", "platypus", "play", "player", "playground", "playroom", "playwright",
"plea", "pleasure", "pleat", "pledge", "plenty", "plier", "pliers", "plight", "plot", "plough", "plover", "plow", "plowman", "plug",
"plugin", "plum", "plumber", "plume", "plunger", "plywood", "pneumonia", "pocket", "pocketbook", "pod", "podcast", "poem",
"poet", "poetry", "poignance", "point", "poison", "poisoning", "poker", "polarisation", "polarization", "pole", "polenta", "police",
"policeman", "policy", "polish", "politician", "politics", "poll", "polliwog", "pollutant", "pollution", "polo", "polyester", "polyp",
"pomegranate", "pomelo", "pompom", "poncho", "pond", "pony", "pool", "poor", "pop", "popcorn", "poppy", "popsicle", "popularity", "population",
"populist", "porcelain", "porch", "porcupine", "pork", "porpoise", "port", "porter", "portfolio", "porthole", "portion", "portrait",
"position", "possession", "possibility", "possible", "post", "postage", "postbox", "poster", "posterior", "postfix", "pot", "potato",
"potential", "pottery", "potty", "pouch", "poultry", "pound", "pounding", "poverty", "powder", "power", "practice", "practitioner", "prairie",
"praise", "pray", "prayer", "precedence", "precedent", "precipitation", "precision", "predecessor", "preface", "preference", "prefix",
"pregnancy", "prejudice", "prelude", "premeditation", "premier", "premise", "premium", "preoccupation", "preparation", "prescription",
"presence", "present", "presentation", "preservation", "preserves", "presidency", "president", "press", "pressroom", "pressure", "pressurisation",
"pressurization", "prestige", "presume", "pretzel", "prevalence", "prevention", "prey", "price", "pricing", "pride", "priest", "priesthood",
"primary", "primate", "prince", "princess", "principal", "principle", "print", "printer", "printing", "prior", "priority", "prison",
"prisoner", "privacy", "private", "privilege", "prize", "prizefight", "probability", "probation", "probe", "problem", "procedure", "proceedings",
"process", "processing", "processor", "proctor", "procurement", "produce", "producer", "product", "production", "productivity", "profession",
"professional", "professor", "profile", "profit", "progenitor", "program", "programme", "programming", "progress", "progression", "prohibition",
"project", "proliferation", "promenade", "promise", "promotion", "prompt", "pronoun", "pronunciation", "proof", "propaganda",
"propane", "property", "prophet", "proponent", "proportion", "proposal", "proposition", "proprietor", "prose", "prosecution", "prosecutor",
"prospect", "prosperity", "prostacyclin", "prostanoid", "prostrate", "protection", "protein", "protest", "protocol", "providence", "provider",
"province", "provision", "prow", "proximal", "proximity", "prune", "pruner", "pseudocode", "pseudoscience", "psychiatrist", "psychoanalyst",
"psychologist", "psychology", "ptarmigan", "pub", "public", "publication", "publicity", "publisher", "publishing", "pudding", "puddle",
"puffin", "pug", "puggle", "pulley", "pulse", "puma", "pump", "pumpernickel", "pumpkin", "pumpkinseed", "pun", "punch", "punctuation",
"punishment", "pup", "pupa", "pupil", "puppet", "puppy", "purchase", "puritan", "purity", "purple", "purpose", "purr", "purse", "pursuit",
"push", "pusher", "put", "puzzle", "pyramid", "pyridine", "quadrant", "quail", "qualification", "quality", "quantity", "quart", "quarter",
"quartet", "quartz", "queen", "query", "quest", "question", "questioner", "questionnaire", "quiche", "quicksand", "quiet", "quill", "quilt",
"quince", "quinoa", "quit", "quiver", "quota", "quotation", "quote", "rabbi", "rabbit", "raccoon", "race", "racer", "racing", "racism",
"racist", "rack", "radar", "radiator", "radio", "radiosonde", "radish", "raffle", "raft", "rag", "rage", "raid", "rail", "railing", "railroad",
"railway", "raiment", "rain", "rainbow", "raincoat", "rainmaker", "rainstorm", "rainy", "raise", "raisin", "rake", "rally", "ram", "rambler",
"ramen", "ramie", "ranch", "rancher", "randomisation", "randomization", "range", "ranger", "rank", "rap", "rape", "raspberry", "rat",
"rate", "ratepayer", "rating", "ratio", "rationale", "rations", "raven", "ravioli", "rawhide", "ray", "rayon", "razor", "reach", "reactant",
"reaction", "read", "reader", "readiness", "reading", "real", "reality", "realization", "realm", "reamer", "rear", "reason", "reasoning",
"rebel", "rebellion", "reboot", "recall", "recapitulation", "receipt", "receiver", "reception", "receptor", "recess", "recession", "recipe",
"recipient", "reciprocity", "reclamation", "recliner", "recognition", "recollection", "recommendation", "reconsideration", "record",
"recorder", "recording", "recovery", "recreation", "recruit", "rectangle", "red", "redesign", "redhead", "redirect", "rediscovery", "reduction",
"reef", "refectory", "reference", "referendum", "reflection", "reform", "refreshments", "refrigerator", "refuge", "refund", "refusal",
"refuse", "regard", "regime", "region", "regionalism", "register", "registration", "registry", "regret", "regulation", "regulator",
"rehospitalization", "reindeer", "reinscription", "reject", "relation", "relationship", "relative", "relaxation", "relay", "release",
"reliability", "relief", "religion", "relish", "reluctance", "remains", "remark", "reminder", "remnant", "remote", "removal", "renaissance",
"rent", "reorganisation", "reorganization", "repair", "reparation", "repayment", "repeat", "replacement", "replica", "replication", "reply",
"report", "reporter", "reporting", "repository", "representation", "representative", "reprocessing", "republic", "republican", "reputation",
"request", "requirement", "resale", "rescue", "research", "researcher", "resemblance", "reservation", "reserve", "reservoir", "reset",
"residence", "resident", "residue", "resist", "resistance", "resolution", "resolve", "resort", "resource", "respect", "respite", "response",
"responsibility", "rest", "restaurant", "restoration", "restriction", "restroom", "restructuring", "result", "resume", "retailer", "retention",
"rethinking", "retina", "retirement", "retouching", "retreat", "retrospect", "retrospective", "retrospectivity", "return", "reunion",
"revascularisation", "revascularization", "reveal", "revelation", "revenant", "revenge", "revenue", "reversal", "reverse", "review",
"revitalisation", "revitalization", "revival", "revolution", "revolver", "reward", "rhetoric", "rheumatism", "rhinoceros", "rhubarb",
"rhyme", "rhythm", "rib", "ribbon", "rice", "riddle", "ride", "rider", "ridge", "riding", "rifle", "right", "rim", "ring", "ringworm",
"riot", "rip", "ripple", "rise", "riser", "risk", "rite", "ritual", "river", "riverbed", "rivulet", "road", "roadway", "roar", "roast",
"robe", "robin", "robot", "robotics", "rock", "rocker", "rocket", "rod", "role", "roll", "roller", "romaine", "romance",
"roof", "room", "roommate", "rooster", "root", "rope", "rose", "rosemary", "roster", "rostrum", "rotation", "round", "roundabout", "route",
"router", "routine", "row", "rowboat", "rowing", "rubber", "rubric", "ruby", "ruckus", "rudiment", "ruffle", "rug", "rugby",
"ruin", "rule", "ruler", "ruling", "rum", "rumor", "run", "runaway", "runner", "running", "runway", "rush", "rust", "rutabaga", "rye",
"sabre", "sac", "sack", "saddle", "sadness", "safari", "safe", "safeguard", "safety", "saffron", "sage", "sail", "sailboat", "sailing",
"sailor", "saint", "sake", "salad", "salami", "salary", "sale", "salesman", "salmon", "salon", "saloon", "salsa", "salt", "salute", "samovar",
"sampan", "sample", "samurai", "sanction", "sanctity", "sanctuary", "sand", "sandal", "sandbar", "sandpaper", "sandwich", "sanity", "sardine",
"sari", "sarong", "sash", "satellite", "satin", "satire", "satisfaction", "sauce", "saucer", "sauerkraut", "sausage", "savage", "savannah",
"saving", "savings", "savior", "saviour", "savory", "saw", "saxophone", "scaffold", "scale", "scallion", "scallops", "scalp", "scam",
"scanner", "scarecrow", "scarf", "scarification", "scenario", "scene", "scenery", "scent", "schedule", "scheduling", "schema", "scheme",
"schizophrenic", "schnitzel", "scholar", "scholarship", "school", "schoolhouse", "schooner", "science", "scientist", "scimitar", "scissors",
"scooter", "scope", "score", "scorn", "scorpion", "scotch", "scout", "scow", "scrambled", "scrap", "scraper", "scratch", "screamer",
"screen", "screening", "screenwriting", "screw", "screwdriver", "scrim", "scrip", "script", "scripture", "scrutiny", "sculpting",
"sculptural", "sculpture", "sea", "seabass", "seafood", "seagull", "seal", "seaplane", "search", "seashore", "seaside", "season", "seat",
"seaweed", "second", "secrecy", "secret", "secretariat", "secretary", "secretion", "section", "sectional", "sector", "security", "sediment",
"seed", "seeder", "seeker", "seep", "segment", "seizure", "selection", "self", "seller",
"selling", "semantics", "semester", "semicircle", "semicolon", "semiconductor", "seminar", "senate", "senator", "sender", "senior", "sense",
"sensibility", "sensitive", "sensitivity", "sensor", "sentence", "sentencing", "sentiment", "sepal", "separation", "septicaemia", "sequel",
"sequence", "serial", "series", "sermon", "serum", "serval", "servant", "server", "service", "servitude", "sesame", "session", "set",
"setback", "setting", "settlement", "settler", "severity", "sewer", "sex", "sexuality", "shack", "shackle", "shade", "shadow", "shadowbox",
"shakedown", "shaker", "shallot", "shallows", "shame", "shampoo", "shanty", "shape", "share", "shareholder", "shark", "shaw", "shawl",
"shear", "shearling", "sheath", "shed", "sheep", "sheet", "shelf", "shell", "shelter", "sherbet", "sherry", "shield", "shift", "shin",
"shine", "shingle", "ship", "shipper", "shipping", "shipyard", "shirt", "shirtdress", "shoat", "shock", "shoe",
"shoehorn", "shoelace", "shoemaker", "shoes", "shoestring", "shofar", "shoot", "shootdown", "shop", "shopper", "shopping", "shore", "shoreline",
"short", "shortage", "shorts", "shortwave", "shot", "shoulder", "shout", "shovel", "show", "shower", "shred", "shrimp",
"shrine", "shutdown", "sibling", "sick", "sickness", "side", "sideboard", "sideburns", "sidecar", "sidestream", "sidewalk", "siding",
"siege", "sigh", "sight", "sightseeing", "sign", "signal", "signature", "signet", "significance", "signify", "signup", "silence", "silica",
"silicon", "silk", "silkworm", "sill", "silly", "silo", "silver", "similarity", "simple", "simplicity", "simplification", "simvastatin",
"sin", "singer", "singing", "singular", "sink", "sinuosity", "sip", "sir", "sister", "sitar", "site", "situation", "size",
"skate", "skating", "skean", "skeleton", "ski", "skiing", "skill", "skin", "skirt", "skull", "skullcap", "skullduggery", "skunk", "sky",
"skylight", "skyline", "skyscraper", "skywalk", "slang", "slapstick", "slash", "slate", "slavery", "slaw", "sled", "sledge",
"sleep", "sleepiness", "sleeping", "sleet", "sleuth", "slice", "slide", "slider", "slime", "slip", "slipper", "slippers", "slope", "slot",
"sloth", "slump", "smell", "smelting", "smile", "smith", "smock", "smog", "smoke", "smoking", "smolt", "smuggling", "snack", "snail",
"snake", "snakebite", "snap", "snarl", "sneaker", "sneakers", "sneeze", "sniffle", "snob", "snorer", "snow", "snowboarding", "snowflake",
"snowman", "snowmobiling", "snowplow", "snowstorm", "snowsuit", "snuck", "snug", "snuggle", "soap", "soccer", "socialism", "socialist",
"society", "sociology", "sock", "socks", "soda", "sofa", "softball", "softdrink", "softening", "software", "soil", "soldier", "sole",
"solicitation", "solicitor", "solidarity", "solidity", "soliloquy", "solitaire", "solution", "solvency", "sombrero", "somebody", "someone",
"someplace", "somersault", "something", "somewhere", "son", "sonar", "sonata", "song", "songbird", "sonnet", "soot", "sophomore", "soprano",
"sorbet", "sorghum", "sorrel", "sorrow", "sort", "soul", "soulmate", "sound", "soundness", "soup", "source", "sourwood", "sousaphone",
"south", "southeast", "souvenir", "sovereignty", "sow", "soy", "soybean", "space", "spacing", "spade", "spaghetti", "span", "spandex",
"spank", "sparerib", "spark", "sparrow", "spasm", "spat", "spatula", "spawn", "speaker", "speakerphone", "speaking", "spear", "spec",
"special", "specialist", "specialty", "species", "specification", "spectacle", "spectacles", "spectrograph", "spectrum", "speculation",
"speech", "speed", "speedboat", "spell", "spelling", "spelt", "spending", "sphere", "sphynx", "spice", "spider", "spiderling", "spike",
"spill", "spinach", "spine", "spiral", "spirit", "spiritual", "spirituality", "spit", "spite", "spleen", "splendor", "split", "spokesman",
"spokeswoman", "sponge", "sponsor", "sponsorship", "spool", "spoon", "spork", "sport", "sportsman", "spot", "spotlight", "spouse", "sprag",
"sprat", "spray", "spread", "spreadsheet", "spree", "spring", "sprinkles", "sprinter", "sprout", "spruce", "spud", "spume", "spur", "spy",
"spyglass", "square", "squash", "squatter", "squeegee", "squid", "squirrel", "stab", "stability", "stable", "stack", "stacking", "stadium",
"staff", "stag", "stage", "stain", "stair", "staircase", "stake", "stalk", "stall", "stallion", "stamen", "stamina", "stamp", "stance",
"stand", "standard", "standardisation", "standardization", "standing", "standoff", "standpoint", "star", "starboard", "start", "starter",
"state", "statement", "statin", "station", "statistic", "statistics", "statue", "status", "statute", "stay", "steak",
"stealth", "steam", "steamroller", "steel", "steeple", "stem", "stench", "stencil", "step",
"stepdaughter", "stepmother",
"stepson", "stereo", "stew", "steward", "stick", "sticker", "stiletto", "still", "stimulation", "stimulus", "sting",
"stinger", "stitch", "stitcher", "stock", "stockings", "stole", "stomach", "stone", "stonework", "stool",
"stop", "stopsign", "stopwatch", "storage", "store", "storey", "storm", "story", "storyboard", "stot", "stove", "strait",
"strand", "stranger", "strap", "strategy", "straw", "strawberry", "strawman", "stream", "street", "streetcar", "strength", "stress",
"stretch", "strife", "strike", "string", "strip", "stripe", "strobe", "stroke", "structure", "strudel", "struggle", "stucco", "stud",
"student", "studio", "study", "stuff", "stumbling", "stump", "stupidity", "sturgeon", "sty", "style", "styling", "stylus", "sub", "subcomponent",
"subconscious", "subcontractor", "subexpression", "subgroup", "subject", "submarine", "submitter", "subprime", "subroutine", "subscription",
"subsection", "subset", "subsidence", "subsidiary", "subsidy", "substance", "substitution", "subtitle", "suburb", "subway", "success",
"succotash", "suck", "sucker", "suede", "suet", "suffocation", "sugar", "suggestion", "suicide", "suit", "suitcase", "suite", "sulfur",
"sultan", "sum", "summary", "summer", "summit", "sun", "sunbeam", "sunbonnet", "sundae", "sunday", "sundial", "sunflower", "sunglasses",
"sunlamp", "sunlight", "sunrise", "sunroom", "sunset", "sunshine", "superiority", "supermarket", "supernatural", "supervision", "supervisor",
"supper", "supplement", "supplier", "supply", "support", "supporter", "suppression", "supreme", "surface", "surfboard", "surge", "surgeon",
"surgery", "surname", "surplus", "surprise", "surround", "surroundings", "surrounds", "survey", "survival", "survivor", "sushi", "suspect",
"suspenders", "suspension", "sustainment", "sustenance", "swallow", "swamp", "swan", "swanling", "swath", "sweat", "sweater", "sweatshirt",
"sweatshop", "sweatsuit", "sweets", "swell", "swim", "swimming", "swimsuit", "swine", "swing", "switch", "switchboard", "switching",
"swivel", "sword", "swordfight", "swordfish", "sycamore", "symbol", "symmetry", "sympathy", "symptom", "syndicate", "syndrome", "synergy",
"synod", "synonym", "synthesis", "syrup", "system", "tab", "tabby", "tabernacle", "table", "tablecloth", "tablet", "tabletop",
"tachometer", "tackle", "taco", "tactics", "tactile", "tadpole", "tag", "tail", "tailbud", "tailor", "tailspin", "takeover",
"tale", "talent", "talk", "talking", "tamale", "tambour", "tambourine", "tan", "tandem", "tangerine", "tank",
"tanker", "tankful", "tap", "tape", "tapioca", "target", "taro", "tarragon", "tart", "task", "tassel", "taste", "tatami", "tattler",
"tattoo", "tavern", "tax", "taxi", "taxicab", "taxpayer", "tea", "teacher", "teaching", "team", "teammate", "teapot", "tear", "tech",
"technician", "technique", "technologist", "technology", "tectonics", "teen", "teenager", "teepee", "telephone", "telescreen", "teletype",
"television", "tell", "teller", "temp", "temper", "temperature", "temple", "tempo", "temporariness", "temporary", "temptation", "temptress",
"tenant", "tendency", "tender", "tenement", "tenet", "tennis", "tenor", "tension", "tensor", "tent", "tentacle", "tenth", "tepee", "teriyaki",
"term", "terminal", "termination", "terminology", "termite", "terrace", "terracotta", "terrapin", "terrarium", "territory", "terror",
"terrorism", "terrorist", "test", "testament", "testimonial", "testimony", "testing", "text", "textbook", "textual", "texture", "thanks",
"thaw", "theater", "theft", "theism", "theme", "theology", "theory", "therapist", "therapy", "thermals", "thermometer", "thermostat",
"thesis", "thickness", "thief", "thigh", "thing", "thinking", "thirst", "thistle", "thong", "thongs", "thorn", "thought", "thousand",
"thread", "threat", "threshold", "thrift", "thrill", "throat", "throne", "thrush", "thrust", "thug", "thumb", "thump", "thunder", "thunderbolt",
"thunderhead", "thunderstorm", "thyme", "tiara", "tic", "tick", "ticket", "tide", "tie", "tiger", "tights", "tile", "till", "tilt", "timbale",
"timber", "time", "timeline", "timeout", "timer", "timetable", "timing", "timpani", "tin", "tinderbox", "tinkle", "tintype", "tip", "tire",
"tissue", "titanium", "title", "toad", "toast", "toaster", "tobacco", "today", "toe", "toenail", "toffee", "tofu", "tog", "toga", "toilet",
"tolerance", "tolerant", "toll", "tomatillo", "tomato", "tomb", "tomography", "tomorrow", "ton", "tonality", "tone", "tongue",
"tonic", "tonight", "tool", "toot", "tooth", "toothbrush", "toothpaste", "toothpick", "top", "topic", "topsail", "toque",
"toreador", "tornado", "torso", "torte", "tortellini", "tortilla", "tortoise", "tosser", "total", "tote", "touch", "tour",
"tourism", "tourist", "tournament", "towel", "tower", "town", "townhouse", "township", "toy", "trace", "trachoma", "track",
"tracking", "tracksuit", "tract", "tractor", "trade", "trader", "trading", "tradition", "traditionalism", "traffic", "trafficker", "tragedy",
"trail", "trailer", "trailpatrol", "train", "trainer", "training", "trait", "tram", "tramp", "trance", "transaction", "transcript", "transfer",
"transformation", "transit", "transition", "translation", "transmission", "transom", "transparency", "transplantation", "transport",
"transportation", "trap", "trapdoor", "trapezium", "trapezoid", "trash", "travel", "traveler", "tray", "treasure", "treasury", "treat",
"treatment", "treaty", "tree", "trek", "trellis", "tremor", "trench", "trend", "triad", "trial", "triangle", "tribe", "tributary", "trick",
"trigger", "trigonometry", "trillion", "trim", "trinket", "trip", "tripod", "tritone", "triumph", "trolley", "trombone", "troop", "trooper",
"trophy", "trouble", "trousers", "trout", "trove", "trowel", "truck", "trumpet", "trunk", "trust", "trustee", "truth", "try", "tsunami",
"tub", "tuba", "tube", "tuber", "tug", "tugboat", "tuition", "tulip", "tumbler", "tummy", "tuna", "tune", "tunic", "tunnel",
"turban", "turf", "turkey", "turmeric", "turn", "turning", "turnip", "turnover", "turnstile", "turret", "turtle", "tusk", "tussle", "tutu",
"tuxedo", "tweet", "tweezers", "twig", "twilight", "twine", "twins", "twist", "twister", "twitter", "type", "typeface", "typewriter",
"typhoon", "ukulele", "ultimatum", "umbrella", "unblinking", "uncertainty", "uncle", "underclothes", "underestimate", "underground",
"underneath", "underpants", "underpass", "undershirt", "understanding", "understatement", "undertaker", "underwear", "underweight", "underwire",
"underwriting", "unemployment", "unibody", "uniform", "uniformity", "union", "unique", "unit", "unity", "universe", "university", "update",
"upgrade", "uplift", "upper", "upstairs", "upward", "urge", "urgency", "urn", "usage", "use", "user", "usher", "usual", "utensil", "utilisation",
"utility", "utilization", "vacation", "vaccine", "vacuum", "vagrant", "valance", "valentine", "validate", "validity", "valley", "valuable",
"value", "vampire", "van", "vanadyl", "vane", "vanilla", "vanity", "variability", "variable", "variant", "variation", "variety", "vascular",
"vase", "vault", "vaulting", "veal", "vector", "vegetable", "vegetarian", "vegetarianism", "vegetation", "vehicle", "veil", "vein", "veldt",
"vellum", "velocity", "velodrome", "velvet", "vendor", "veneer", "vengeance", "venison", "venom", "venti", "venture", "venue", "veranda",
"verb", "verdict", "verification", "vermicelli", "vernacular", "verse", "version", "vertigo", "verve", "vessel", "vest", "vestment",
"vet", "veteran", "veterinarian", "veto", "viability", "vibe", "vibraphone", "vibration", "vibrissae", "vice", "vicinity", "victim",
"victory", "video", "view", "viewer", "vignette", "villa", "village", "vine", "vinegar", "vineyard", "vintage", "vintner", "vinyl", "viola",
"violation", "violence", "violet", "violin", "virginal", "virtue", "virus", "visa", "viscose", "vise", "vision", "visit", "visitor",
"visor", "vista", "visual", "vitality", "vitamin", "vitro", "vivo", "vixen", "vodka", "vogue", "voice", "void", "vol", "volatility",
"volcano", "volleyball", "volume", "volunteer", "volunteering", "vomit", "vote", "voter", "voting", "voyage", "vulture", "wad", "wafer",
"waffle", "wage", "wagon", "waist", "waistband", "wait", "waiter", "waiting", "waitress", "waiver", "wake", "walk", "walker", "walking",
"walkway", "wall", "wallaby", "wallet", "walnut", "walrus", "wampum", "wannabe", "want", "war", "warden", "wardrobe", "warfare", "warlock",
"warlord", "warming", "warmth", "warning", "warrant", "warren", "warrior", "wasabi", "wash", "washbasin", "washcloth", "washer",
"washtub", "wasp", "waste", "wastebasket", "wasting", "watch", "watcher", "watchmaker", "water", "waterbed", "watercress", "waterfall",
"waterfront", "watermelon", "waterskiing", "waterspout", "waterwheel", "wave", "waveform", "wax", "way", "weakness", "wealth", "weapon",
"wear", "weasel", "weather", "web", "webinar", "webmail", "webpage", "website", "wedding", "wedge", "weed", "weeder", "weedkiller", "week",
"weekend", "weekender", "weight", "weird", "welcome", "welfare", "well", "west", "western", "wetland", "wetsuit",
"whack", "whale", "wharf", "wheat", "wheel", "whelp", "whey", "whip", "whirlpool", "whirlwind", "whisker", "whiskey", "whisper", "whistle",
"white", "whole", "wholesale", "wholesaler", "whorl", "wick", "widget", "widow", "width", "wife", "wifi", "wild", "wildebeest", "wilderness",
"wildlife", "will", "willingness", "willow", "win", "wind", "windage", "window", "windscreen", "windshield", "wine", "winery",
"wing", "wingman", "wingtip", "wink", "winner", "winter", "wire", "wiretap", "wiring", "wisdom", "wiseguy", "wish", "wisteria", "wit",
"witch", "withdrawal", "witness", "wok", "wolf", "woman", "wombat", "wonder", "wont", "wood", "woodchuck", "woodland",
"woodshed", "woodwind", "wool", "woolens", "word", "wording", "work", "workbench", "worker", "workforce", "workhorse", "working", "workout",
"workplace", "workshop", "world", "worm", "worry", "worship", "worshiper", "worth", "wound", "wrap", "wraparound", "wrapper", "wrapping",
"wreck", "wrecker", "wren", "wrench", "wrestler", "wriggler", "wrinkle", "wrist", "writer", "writing", "wrong", "xylophone", "yacht",
"yahoo", "yak", "yam", "yang", "yard", "yarmulke", "yarn", "yawl", "year", "yeast", "yellow", "yellowjacket", "yesterday", "yew", "yin",
"yoga", "yogurt", "yoke", "yolk", "young", "youngster", "yourself", "youth", "yoyo", "yurt", "zampone", "zebra", "zebrafish", "zen",
"zephyr", "zero", "ziggurat", "zinc", "zipper", "zither", "zombie", "zone", "zoo", "zoologist", "zoology", "zucchini"
};
std::string_view obfuscateWord(std::string_view src, WordMap & obfuscate_map, WordSet & used_nouns, SipHash hash_func)
{
/// Prevent using too many nouns
if (obfuscate_map.size() * 2 > nouns.size())
throw Exception("Too many unique identifiers in queries", ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS);
std::string_view & mapped = obfuscate_map[src];
if (!mapped.empty())
return mapped;
hash_func.update(src.data(), src.size());
std::string_view noun = nouns.begin()[hash_func.get64() % nouns.size()];
/// Prevent collisions
while (!used_nouns.insert(noun).second)
{
hash_func.update('\0');
noun = nouns.begin()[hash_func.get64() % nouns.size()];
}
mapped = noun;
return mapped;
}
void obfuscateIdentifier(std::string_view src, WriteBuffer & result, WordMap & obfuscate_map, WordSet & used_nouns, SipHash hash_func)
{
/// Find words in form 'snake_case', 'CamelCase' or 'ALL_CAPS'.
const char * src_pos = src.data();
const char * src_end = src_pos + src.size();
const char * word_begin = src_pos;
bool word_has_alphanumerics = false;
auto append_word = [&]
{
std::string_view word(word_begin, src_pos - word_begin);
if (keep_words.count(word))
{
result.write(word.data(), word.size());
}
else
{
std::string_view obfuscated_word = obfuscateWord(word, obfuscate_map, used_nouns, hash_func);
/// Match the style of source word.
bool first_caps = !word.empty() && isUpperAlphaASCII(word[0]);
bool all_caps = first_caps && word.size() >= 2 && isUpperAlphaASCII(word[1]);
for (size_t i = 0, size = obfuscated_word.size(); i < size; ++i)
{
if (all_caps || (i == 0 && first_caps))
result.write(toUpperIfAlphaASCII(obfuscated_word[i]));
else
result.write(obfuscated_word[i]);
}
}
word_begin = src_pos;
word_has_alphanumerics = false;
};
while (src_pos < src_end)
{
if (isAlphaNumericASCII(src_pos[0]))
word_has_alphanumerics = true;
if (word_has_alphanumerics && src_pos[0] == '_')
{
append_word();
result.write('_');
++word_begin;
}
else if (word_has_alphanumerics && isUpperAlphaASCII(src_pos[0]) && isLowerAlphaASCII(src_pos[-1])) /// xX
{
append_word();
}
++src_pos;
}
if (word_begin < src_pos)
append_word();
}
void obfuscateLiteral(std::string_view src, WriteBuffer & result, SipHash hash_func)
{
const char * src_pos = src.data();
const char * src_end = src_pos + src.size();
while (src_pos < src_end)
{
/// Date
if (src_pos + strlen("0000-00-00") <= src_end
&& isNumericASCII(src_pos[0])
&& isNumericASCII(src_pos[1])
&& isNumericASCII(src_pos[2])
&& isNumericASCII(src_pos[3])
&& src_pos[4] == '-'
&& isNumericASCII(src_pos[5])
&& isNumericASCII(src_pos[6])
&& src_pos[7] == '-'
&& isNumericASCII(src_pos[8])
&& isNumericASCII(src_pos[9]))
{
DayNum date;
ReadBufferFromMemory in(src_pos, strlen("0000-00-00"));
readDateText(date, in);
SipHash hash_func_date = hash_func;
if (date != 0)
{
date += hash_func_date.get64() % 256;
}
writeDateText(date, result);
src_pos += strlen("0000-00-00");
/// DateTime
if (src_pos + strlen(" 00:00:00") <= src_end
&& isNumericASCII(src_pos[1])
&& isNumericASCII(src_pos[2])
&& src_pos[3] == ':'
&& isNumericASCII(src_pos[4])
&& isNumericASCII(src_pos[5])
&& src_pos[6] == ':'
&& isNumericASCII(src_pos[7])
&& isNumericASCII(src_pos[8]))
{
result.write(src_pos[0]);
hash_func_date.update(src_pos + 1, strlen("00:00:00"));
uint64_t hash_value = hash_func_date.get64();
uint32_t new_hour = hash_value % 24;
hash_value /= 24;
uint32_t new_minute = hash_value % 60;
hash_value /= 60;
uint32_t new_second = hash_value % 60;
result.write('0' + (new_hour / 10));
result.write('0' + (new_hour % 10));
result.write(':');
result.write('0' + (new_minute / 10));
result.write('0' + (new_minute % 10));
result.write(':');
result.write('0' + (new_second / 10));
result.write('0' + (new_second % 10));
src_pos += strlen(" 00:00:00");
}
}
else if (isNumericASCII(src_pos[0]))
{
/// Number
if (src_pos[0] == '0' || src_pos[0] == '1')
{
/// Keep zero and one as is.
result.write(src_pos[0]);
++src_pos;
}
else
{
ReadBufferFromMemory in(src_pos, src_end - src_pos);
uint64_t num;
readIntText(num, in);
SipHash hash_func_num = hash_func;
hash_func_num.update(src_pos, in.count());
src_pos += in.count();
/// Obfuscate number but keep it within same power of two range.
uint64_t obfuscated = hash_func_num.get64();
uint64_t log2 = bitScanReverse(num);
obfuscated = (1ULL << log2) + obfuscated % (1ULL << log2);
writeIntText(obfuscated, result);
}
}
else if (src_pos + 1 < src_end
&& (src_pos[0] == 'e' || src_pos[0] == 'E')
&& (isNumericASCII(src_pos[1]) || (src_pos[1] == '-' && src_pos + 2 < src_end && isNumericASCII(src_pos[2]))))
{
/// Something like an exponent of floating point number. Keep it as is.
/// But if it looks like a large number, overflow it into 16 bit.
result.write(src_pos[0]);
++src_pos;
ReadBufferFromMemory in(src_pos, src_end - src_pos);
int16_t num;
readIntText(num, in);
writeIntText(num, result);
src_pos += in.count();
}
else if (isAlphaASCII(src_pos[0]))
{
/// Alphabetial characters
const char * alpha_end = src_pos + 1;
while (alpha_end < src_end && isAlphaASCII(*alpha_end))
++alpha_end;
hash_func.update(src_pos, alpha_end - src_pos);
pcg64 rng(hash_func.get64());
while (src_pos < alpha_end)
{
auto random = rng();
if (isLowerAlphaASCII(*src_pos))
result.write('a' + random % 26);
else
result.write('A' + random % 26);
++src_pos;
}
}
else if (isASCII(src_pos[0]))
{
/// Punctuation, whitespace and control characters - keep as is.
result.write(src_pos[0]);
++src_pos;
}
else if (src_pos[0] <= '\xBF')
{
/// Continuation of UTF-8 sequence.
hash_func.update(src_pos[0]);
uint64_t hash = hash_func.get64();
char c = 0x80 + hash % (0xC0 - 0x80);
result.write(c);
++src_pos;
}
else
{
/// Start of UTF-8 sequence.
hash_func.update(src_pos[0]);
uint64_t hash = hash_func.get64();
if (src_pos[0] < '\xE0')
{
char c = 0xC0 + hash % 32;
result.write(c);
}
else if (src_pos[0] < '\xF0')
{
char c = 0xE0 + hash % 16;
result.write(c);
}
else
{
char c = 0xF0 + hash % 8;
result.write(c);
}
++src_pos;
}
}
}
}
void obfuscateQueries(
std::string_view src,
WriteBuffer & result,
WordMap & obfuscate_map,
WordSet & used_nouns,
SipHash hash_func,
KnownIdentifierFunc known_identifier_func)
{
Lexer lexer(src.data(), src.data() + src.size());
while (true)
{
Token token = lexer.nextToken();
std::string_view whole_token(token.begin, token.size());
if (token.isEnd())
break;
if (token.type == TokenType::BareWord)
{
std::string whole_token_uppercase(whole_token);
Poco::toUpperInPlace(whole_token_uppercase);
if (keywords.count(whole_token_uppercase)
|| known_identifier_func(whole_token))
{
/// Keep keywords as is.
result.write(token.begin, token.size());
}
else
{
/// Obfuscate identifiers
obfuscateIdentifier(whole_token, result, obfuscate_map, used_nouns, hash_func);
}
}
else if (token.type == TokenType::QuotedIdentifier)
{
assert(token.size() >= 2);
/// Write quotes and the obfuscated content inside.
result.write(*token.begin);
obfuscateIdentifier({token.begin + 1, token.size() - 2}, result, obfuscate_map, used_nouns, hash_func);
result.write(token.end[-1]);
}
else if (token.type == TokenType::Number)
{
obfuscateLiteral(whole_token, result, hash_func);
}
else if (token.type == TokenType::StringLiteral)
{
assert(token.size() >= 2);
result.write(*token.begin);
obfuscateLiteral({token.begin + 1, token.size() - 2}, result, hash_func);
result.write(token.end[-1]);
}
else if (token.type == TokenType::Comment)
{
/// Skip comments - they may contain confidential info.
}
else
{
/// Everyting else is kept as is.
result.write(token.begin, token.size());
}
}
}
}

View File

@ -0,0 +1,50 @@
#pragma once
#include <string>
#include <unordered_set>
#include <unordered_map>
#include <string_view>
#include <functional>
#include <Common/SipHash.h>
namespace DB
{
class WriteBuffer;
using WordMap = std::unordered_map<std::string_view, std::string_view>;
using WordSet = std::unordered_set<std::string_view>;
using KnownIdentifierFunc = std::function<bool(std::string_view)>;
/** Takes one or multiple queries and obfuscates them by replacing identifiers to pseudorandom words
* and replacing literals to random values, while preserving the structure of the queries and the general sense.
*
* Its intended use case is when the user wants to share their queries for testing and debugging
* but is afraid to disclose the details about their column names, domain area and values of constants.
*
* It can obfuscate multiple queries in consistent fashion - identical names will be transformed to identical results.
*
* The function is not guaranteed to always give correct result or to be secure. It's implemented in "best effort" fashion.
*
* @param src - a string with source queries.
* @param result - where the obfuscated queries will be written.
* @param obfuscate_map - information about substituted identifiers
* (pass empty map at the beginning or reuse it from previous invocation to get consistent result)
* @param used_nouns - information about words used for substitution
* (pass empty set at the beginning or reuse it from previous invocation to get consistent result)
* @param hash_func - hash function that will be used as a pseudorandom source,
* it's recommended to preseed the function before passing here.
* @param known_identifier_func - a function that returns true if identifier is known name
* (of function, aggregate function, etc. that should be kept as is). If it returns false, identifier will be obfuscated.
*/
void obfuscateQueries(
std::string_view src,
WriteBuffer & result,
WordMap & obfuscate_map,
WordSet & used_nouns,
SipHash hash_func,
KnownIdentifierFunc known_identifier_func);
}

View File

@ -85,6 +85,7 @@ SRCS(
MySQL/ASTDeclareReference.cpp
MySQL/ASTDeclareSubPartition.cpp
MySQL/ASTDeclareTableOptions.cpp
obfuscateQueries.cpp
parseDatabaseAndTableName.cpp
parseIdentifierOrStringLiteral.cpp
parseIntervalKind.cpp

View File

@ -10,7 +10,6 @@ ForkProcessor::Status ForkProcessor::prepare()
/// Check can output.
bool all_finished = true;
bool all_can_push = true;
size_t num_active_outputs = 0;
@ -18,7 +17,6 @@ ForkProcessor::Status ForkProcessor::prepare()
{
if (!output.isFinished())
{
all_finished = false;
++num_active_outputs;
/// The order is important.
@ -27,7 +25,7 @@ ForkProcessor::Status ForkProcessor::prepare()
}
}
if (all_finished)
if (0 == num_active_outputs)
{
input.close();
return Status::Finished;

View File

@ -44,7 +44,7 @@ public:
/// Information about different TTLs for part. Can be used by
/// TTLSelector to assign merges with TTL.
const MergeTreeDataPartTTLInfos * ttl_infos;
const MergeTreeDataPartTTLInfos * ttl_infos = nullptr;
/// Part compression codec definition.
ASTPtr compression_codec_desc;

View File

@ -717,6 +717,8 @@ protected:
bool require_part_metadata;
/// Relative path data, changes during rename for ordinary databases use
/// under lockForShare if rename is possible.
String relative_data_path;

View File

@ -223,7 +223,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts(
if (part_column_queried)
VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, context);
std::multiset<String> part_values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_part");
auto part_values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_part");
metadata_snapshot->check(real_column_names, data.getVirtuals(), data.getStorageID());

View File

@ -56,10 +56,12 @@ void ReplicatedMergeTreeCleanupThread::run()
void ReplicatedMergeTreeCleanupThread::iterate()
{
storage.clearOldPartsAndRemoveFromZK();
storage.clearOldWriteAheadLogs();
{
auto lock = storage.lockForShare(RWLockImpl::NO_QUERY, storage.getSettings()->lock_acquire_timeout_for_background_operations);
/// Both use relative_data_path which changes during rename, so we
/// do it under share lock
storage.clearOldWriteAheadLogs();
storage.clearOldTemporaryDirectories();
}

View File

@ -92,6 +92,12 @@ String StorageDictionary::generateNamesAndTypesDescription(const NamesAndTypesLi
return ss.str();
}
String StorageDictionary::resolvedDictionaryName() const
{
if (location == Location::SameDatabaseAndNameAsDictionary)
return dictionary_name;
return DatabaseCatalog::instance().resolveDictionaryName(dictionary_name);
}
StorageDictionary::StorageDictionary(
const StorageID & table_id_,
@ -132,7 +138,7 @@ Pipe StorageDictionary::read(
const size_t max_block_size,
const unsigned /*threads*/)
{
auto dictionary = context.getExternalDictionariesLoader().getDictionary(dictionary_name);
auto dictionary = context.getExternalDictionariesLoader().getDictionary(resolvedDictionaryName());
auto stream = dictionary->getBlockInputStream(column_names, max_block_size);
/// TODO: update dictionary interface for processors.
return Pipe(std::make_shared<SourceFromInputStream>(stream));
@ -152,7 +158,8 @@ void registerStorageDictionary(StorageFactory & factory)
if (!args.attach)
{
const auto & dictionary = args.context.getExternalDictionariesLoader().getDictionary(dictionary_name);
auto resolved = DatabaseCatalog::instance().resolveDictionaryName(dictionary_name);
const auto & dictionary = args.context.getExternalDictionariesLoader().getDictionary(resolved);
const DictionaryStructure & dictionary_structure = dictionary->getStructure();
checkNamesAndTypesCompatibleWithDictionary(dictionary_name, args.columns, dictionary_structure);
}

View File

@ -29,6 +29,7 @@ public:
static String generateNamesAndTypesDescription(const NamesAndTypesList & list);
const String & dictionaryName() const { return dictionary_name; }
String resolvedDictionaryName() const;
/// Specifies where the table is located relative to the dictionary.
enum class Location

View File

@ -52,7 +52,6 @@ namespace ErrorCodes
extern const int UNKNOWN_IDENTIFIER;
extern const int INCORRECT_FILE_NAME;
extern const int FILE_DOESNT_EXIST;
extern const int TIMEOUT_EXCEEDED;
}
namespace
@ -200,17 +199,6 @@ StorageFile::StorageFile(CommonArguments args)
setInMemoryMetadata(storage_metadata);
}
static std::chrono::seconds getLockTimeout(const Context & context)
{
const Settings & settings = context.getSettingsRef();
Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds();
if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout)
lock_timeout = settings.max_execution_time.totalSeconds();
return std::chrono::seconds{lock_timeout};
}
class StorageFileSource : public SourceWithProgress
{
public:
@ -257,9 +245,7 @@ public:
{
if (storage->use_table_fd)
{
unique_lock = std::unique_lock(storage->rwlock, getLockTimeout(context));
if (!unique_lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
unique_lock = std::unique_lock(storage->rwlock);
/// We could use common ReadBuffer and WriteBuffer in storage to leverage cache
/// and add ability to seek unseekable files, but cache sync isn't supported.
@ -278,9 +264,7 @@ public:
}
else
{
shared_lock = std::shared_lock(storage->rwlock, getLockTimeout(context));
if (!shared_lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
shared_lock = std::shared_lock(storage->rwlock);
}
}
@ -389,8 +373,8 @@ private:
bool finished_generate = false;
std::shared_lock<std::shared_timed_mutex> shared_lock;
std::unique_lock<std::shared_timed_mutex> unique_lock;
std::shared_lock<std::shared_mutex> shared_lock;
std::unique_lock<std::shared_mutex> unique_lock;
};
@ -433,7 +417,7 @@ Pipe StorageFile::read(
for (size_t i = 0; i < num_streams; ++i)
pipes.emplace_back(std::make_shared<StorageFileSource>(
this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns().getDefaults()));
this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns().getDefaults()));
return Pipe::unitePipes(std::move(pipes));
}
@ -445,16 +429,12 @@ public:
explicit StorageFileBlockOutputStream(
StorageFile & storage_,
const StorageMetadataPtr & metadata_snapshot_,
std::unique_lock<std::shared_timed_mutex> && lock_,
const CompressionMethod compression_method,
const Context & context)
: storage(storage_)
, metadata_snapshot(metadata_snapshot_)
, lock(std::move(lock_))
, lock(storage.rwlock)
{
if (!lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
std::unique_ptr<WriteBufferFromFileDescriptor> naked_buffer = nullptr;
if (storage.use_table_fd)
{
@ -508,7 +488,7 @@ public:
private:
StorageFile & storage;
StorageMetadataPtr metadata_snapshot;
std::unique_lock<std::shared_timed_mutex> lock;
std::unique_lock<std::shared_mutex> lock;
std::unique_ptr<WriteBuffer> write_buf;
BlockOutputStreamPtr writer;
bool prefix_written{false};
@ -526,7 +506,7 @@ BlockOutputStreamPtr StorageFile::write(
if (!paths.empty())
path = paths[0];
return std::make_shared<StorageFileBlockOutputStream>(*this, metadata_snapshot, std::unique_lock{rwlock, getLockTimeout(context)},
return std::make_shared<StorageFileBlockOutputStream>(*this, metadata_snapshot,
chooseCompressionMethod(path, compression_method), context);
}
@ -549,6 +529,8 @@ void StorageFile::rename(const String & new_path_to_table_data, const StorageID
if (path_new == paths[0])
return;
std::unique_lock<std::shared_mutex> lock(rwlock);
Poco::File(Poco::Path(path_new).parent()).createDirectories();
Poco::File(paths[0]).renameTo(path_new);
@ -565,6 +547,8 @@ void StorageFile::truncate(
if (paths.size() != 1)
throw Exception("Can't truncate table '" + getStorageID().getNameForLogs() + "' in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED);
std::unique_lock<std::shared_mutex> lock(rwlock);
if (use_table_fd)
{
if (0 != ::ftruncate(table_fd, 0))

View File

@ -89,7 +89,7 @@ private:
std::atomic<bool> table_fd_was_used{false}; /// To detect repeating reads from stdin
off_t table_fd_init_offset = -1; /// Initial position of fd, used for repeating reads
mutable std::shared_timed_mutex rwlock;
mutable std::shared_mutex rwlock;
Poco::Logger * log = &Poco::Logger::get("StorageFile");
};

View File

@ -39,7 +39,6 @@ namespace DB
namespace ErrorCodes
{
extern const int TIMEOUT_EXCEEDED;
extern const int LOGICAL_ERROR;
extern const int DUPLICATE_COLUMN;
extern const int SIZES_OF_MARKS_FILES_ARE_INCONSISTENT;
@ -51,6 +50,7 @@ namespace ErrorCodes
class LogSource final : public SourceWithProgress
{
public:
static Block getHeader(const NamesAndTypesList & columns)
{
Block res;
@ -116,16 +116,13 @@ private:
class LogBlockOutputStream final : public IBlockOutputStream
{
public:
explicit LogBlockOutputStream(
StorageLog & storage_, const StorageMetadataPtr & metadata_snapshot_, std::unique_lock<std::shared_timed_mutex> && lock_)
explicit LogBlockOutputStream(StorageLog & storage_, const StorageMetadataPtr & metadata_snapshot_)
: storage(storage_)
, metadata_snapshot(metadata_snapshot_)
, lock(std::move(lock_))
, lock(storage.rwlock)
, marks_stream(
storage.disk->writeFile(storage.marks_file_path, 4096, WriteMode::Rewrite))
{
if (!lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
}
~LogBlockOutputStream() override
@ -152,7 +149,7 @@ public:
private:
StorageLog & storage;
StorageMetadataPtr metadata_snapshot;
std::unique_lock<std::shared_timed_mutex> lock;
std::unique_lock<std::shared_mutex> lock;
bool done = false;
struct Stream
@ -510,11 +507,9 @@ void StorageLog::addFiles(const String & column_name, const IDataType & type)
}
void StorageLog::loadMarks(std::chrono::seconds lock_timeout)
void StorageLog::loadMarks()
{
std::unique_lock lock(rwlock, lock_timeout);
if (!lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
std::unique_lock<std::shared_mutex> lock(rwlock);
if (loaded_marks)
return;
@ -557,6 +552,8 @@ void StorageLog::rename(const String & new_path_to_table_data, const StorageID &
{
assert(table_path != new_path_to_table_data);
{
std::unique_lock<std::shared_mutex> lock(rwlock);
disk->moveDirectory(table_path, new_path_to_table_data);
table_path = new_path_to_table_data;
@ -572,6 +569,8 @@ void StorageLog::rename(const String & new_path_to_table_data, const StorageID &
void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &)
{
std::shared_lock<std::shared_mutex> lock(rwlock);
files.clear();
file_count = 0;
loaded_marks = false;
@ -611,17 +610,6 @@ const StorageLog::Marks & StorageLog::getMarksWithRealRowCount(const StorageMeta
return it->second.marks;
}
static std::chrono::seconds getLockTimeout(const Context & context)
{
const Settings & settings = context.getSettingsRef();
Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds();
if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout)
lock_timeout = settings.max_execution_time.totalSeconds();
return std::chrono::seconds{lock_timeout};
}
Pipe StorageLog::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
@ -632,15 +620,11 @@ Pipe StorageLog::read(
unsigned num_streams)
{
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
auto lock_timeout = getLockTimeout(context);
loadMarks(lock_timeout);
loadMarks();
NamesAndTypesList all_columns = Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names));
std::shared_lock lock(rwlock, lock_timeout);
if (!lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
std::shared_lock<std::shared_mutex> lock(rwlock);
Pipes pipes;
@ -669,28 +653,18 @@ Pipe StorageLog::read(
max_read_buffer_size));
}
/// No need to hold lock while reading because we read fixed range of data that does not change while appending more data.
return Pipe::unitePipes(std::move(pipes));
}
BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context)
BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/)
{
auto lock_timeout = getLockTimeout(context);
loadMarks(lock_timeout);
std::unique_lock lock(rwlock, lock_timeout);
if (!lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
return std::make_shared<LogBlockOutputStream>(*this, metadata_snapshot, std::move(lock));
loadMarks();
return std::make_shared<LogBlockOutputStream>(*this, metadata_snapshot);
}
CheckResults StorageLog::checkData(const ASTPtr & /* query */, const Context & context)
CheckResults StorageLog::checkData(const ASTPtr & /* query */, const Context & /* context */)
{
std::shared_lock lock(rwlock, getLockTimeout(context));
if (!lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
std::shared_lock<std::shared_mutex> lock(rwlock);
return file_checker.check();
}

View File

@ -83,7 +83,7 @@ private:
DiskPtr disk;
String table_path;
mutable std::shared_timed_mutex rwlock;
mutable std::shared_mutex rwlock;
Files files;
@ -104,7 +104,7 @@ private:
/// Read marks files if they are not already read.
/// It is done lazily, so that with a large number of tables, the server starts quickly.
/// You can not call with a write locked `rwlock`.
void loadMarks(std::chrono::seconds lock_timeout);
void loadMarks();
/** For normal columns, the number of rows in the block is specified in the marks.
* For array columns and nested structures, there are more than one group of marks that correspond to different files

View File

@ -919,11 +919,13 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask()
{
{
auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations);
/// All use relative_data_path which changes during rename
/// so execute under share lock.
clearOldPartsFromFilesystem();
clearOldTemporaryDirectories();
clearOldWriteAheadLogs();
}
clearOldMutations();
clearOldWriteAheadLogs();
}
///TODO: read deduplicate option from table config

View File

@ -96,7 +96,7 @@ Pipe StorageMySQL::read(
/// TODO: rewrite MySQLBlockInputStream
return Pipe(std::make_shared<SourceFromInputStream>(
std::make_shared<MySQLBlockInputStream>(pool.get(), query, sample_block, max_block_size_)));
std::make_shared<MySQLLazyBlockInputStream>(pool, query, sample_block, max_block_size_, /* auto_close = */ true)));
}

View File

@ -47,13 +47,13 @@ namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int INCORRECT_FILE_NAME;
extern const int TIMEOUT_EXCEEDED;
}
class StripeLogSource final : public SourceWithProgress
{
public:
static Block getHeader(
StorageStripeLog & storage,
const StorageMetadataPtr & metadata_snapshot,
@ -157,11 +157,10 @@ private:
class StripeLogBlockOutputStream final : public IBlockOutputStream
{
public:
explicit StripeLogBlockOutputStream(
StorageStripeLog & storage_, const StorageMetadataPtr & metadata_snapshot_, std::unique_lock<std::shared_timed_mutex> && lock_)
explicit StripeLogBlockOutputStream(StorageStripeLog & storage_, const StorageMetadataPtr & metadata_snapshot_)
: storage(storage_)
, metadata_snapshot(metadata_snapshot_)
, lock(std::move(lock_))
, lock(storage.rwlock)
, data_out_file(storage.table_path + "data.bin")
, data_out_compressed(storage.disk->writeFile(data_out_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append))
, data_out(std::make_unique<CompressedWriteBuffer>(
@ -171,8 +170,6 @@ public:
, index_out(std::make_unique<CompressedWriteBuffer>(*index_out_compressed))
, block_out(*data_out, 0, metadata_snapshot->getSampleBlock(), false, index_out.get(), storage.disk->getFileSize(data_out_file))
{
if (!lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
}
~StripeLogBlockOutputStream() override
@ -226,7 +223,7 @@ public:
private:
StorageStripeLog & storage;
StorageMetadataPtr metadata_snapshot;
std::unique_lock<std::shared_timed_mutex> lock;
std::unique_lock<std::shared_mutex> lock;
String data_out_file;
std::unique_ptr<WriteBuffer> data_out_compressed;
@ -289,6 +286,8 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora
{
assert(table_path != new_path_to_table_data);
{
std::unique_lock<std::shared_mutex> lock(rwlock);
disk->moveDirectory(table_path, new_path_to_table_data);
table_path = new_path_to_table_data;
@ -298,16 +297,6 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora
}
static std::chrono::seconds getLockTimeout(const Context & context)
{
const Settings & settings = context.getSettingsRef();
Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds();
if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout)
lock_timeout = settings.max_execution_time.totalSeconds();
return std::chrono::seconds{lock_timeout};
}
Pipe StorageStripeLog::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
@ -317,9 +306,7 @@ Pipe StorageStripeLog::read(
const size_t /*max_block_size*/,
unsigned num_streams)
{
std::shared_lock lock(rwlock, getLockTimeout(context));
if (!lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
std::shared_lock<std::shared_mutex> lock(rwlock);
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
@ -358,28 +345,24 @@ Pipe StorageStripeLog::read(
}
BlockOutputStreamPtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context)
BlockOutputStreamPtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/)
{
std::unique_lock lock(rwlock, getLockTimeout(context));
if (!lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
return std::make_shared<StripeLogBlockOutputStream>(*this, metadata_snapshot, std::move(lock));
return std::make_shared<StripeLogBlockOutputStream>(*this, metadata_snapshot);
}
CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Context & context)
CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Context & /* context */)
{
std::shared_lock lock(rwlock, getLockTimeout(context));
if (!lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
std::shared_lock<std::shared_mutex> lock(rwlock);
return file_checker.check();
}
void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &)
{
std::shared_lock<std::shared_mutex> lock(rwlock);
disk->clearDirectory(table_path);
file_checker = FileChecker{disk, table_path + "sizes.json"};
}

View File

@ -67,7 +67,7 @@ private:
size_t max_compress_block_size;
FileChecker file_checker;
mutable std::shared_timed_mutex rwlock;
mutable std::shared_mutex rwlock;
Poco::Logger * log;
};

View File

@ -13,7 +13,6 @@
#include <IO/ReadBufferFromFileBase.h>
#include <IO/WriteBufferFromFileBase.h>
#include <IO/LimitReadBuffer.h>
#include <Compression/CompressionFactory.h>
#include <Compression/CompressedReadBuffer.h>
#include <Compression/CompressedWriteBuffer.h>
@ -47,7 +46,6 @@ namespace DB
namespace ErrorCodes
{
extern const int TIMEOUT_EXCEEDED;
extern const int DUPLICATE_COLUMN;
extern const int INCORRECT_FILE_NAME;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
@ -57,6 +55,7 @@ namespace ErrorCodes
class TinyLogSource final : public SourceWithProgress
{
public:
static Block getHeader(const NamesAndTypesList & columns)
{
Block res;
@ -67,17 +66,10 @@ public:
return Nested::flatten(res);
}
TinyLogSource(
size_t block_size_,
const NamesAndTypesList & columns_,
StorageTinyLog & storage_,
size_t max_read_buffer_size_,
FileChecker::Map file_sizes_)
TinyLogSource(size_t block_size_, const NamesAndTypesList & columns_, StorageTinyLog & storage_, size_t max_read_buffer_size_)
: SourceWithProgress(getHeader(columns_))
, block_size(block_size_), columns(columns_), storage(storage_)
, max_read_buffer_size(max_read_buffer_size_), file_sizes(std::move(file_sizes_))
{
}
, block_size(block_size_), columns(columns_), storage(storage_), lock(storage_.rwlock)
, max_read_buffer_size(max_read_buffer_size_) {}
String getName() const override { return "TinyLog"; }
@ -88,21 +80,19 @@ private:
size_t block_size;
NamesAndTypesList columns;
StorageTinyLog & storage;
std::shared_lock<std::shared_mutex> lock;
bool is_finished = false;
size_t max_read_buffer_size;
FileChecker::Map file_sizes;
struct Stream
{
Stream(const DiskPtr & disk, const String & data_path, size_t max_read_buffer_size_, size_t file_size)
Stream(const DiskPtr & disk, const String & data_path, size_t max_read_buffer_size_)
: plain(disk->readFile(data_path, std::min(max_read_buffer_size_, disk->getFileSize(data_path)))),
limited(std::make_unique<LimitReadBuffer>(*plain, file_size, false)),
compressed(*plain)
{
}
std::unique_ptr<ReadBuffer> plain;
std::unique_ptr<ReadBuffer> limited;
CompressedReadBuffer compressed;
};
@ -120,14 +110,9 @@ private:
class TinyLogBlockOutputStream final : public IBlockOutputStream
{
public:
explicit TinyLogBlockOutputStream(
StorageTinyLog & storage_,
const StorageMetadataPtr & metadata_snapshot_,
std::unique_lock<std::shared_timed_mutex> && lock_)
: storage(storage_), metadata_snapshot(metadata_snapshot_), lock(std::move(lock_))
explicit TinyLogBlockOutputStream(StorageTinyLog & storage_, const StorageMetadataPtr & metadata_snapshot_)
: storage(storage_), metadata_snapshot(metadata_snapshot_), lock(storage_.rwlock)
{
if (!lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
}
~TinyLogBlockOutputStream() override
@ -155,7 +140,7 @@ public:
private:
StorageTinyLog & storage;
StorageMetadataPtr metadata_snapshot;
std::unique_lock<std::shared_timed_mutex> lock;
std::unique_lock<std::shared_mutex> lock;
bool done = false;
struct Stream
@ -246,17 +231,13 @@ void TinyLogSource::readData(const String & name, const IDataType & type, IColum
String stream_name = IDataType::getFileNameForStream(name, path);
if (!streams.count(stream_name))
{
String file_path = storage.files[stream_name].data_file_path;
streams[stream_name] = std::make_unique<Stream>(
storage.disk, file_path, max_read_buffer_size, file_sizes[fileName(file_path)]);
}
streams[stream_name] = std::make_unique<Stream>(storage.disk, storage.files[stream_name].data_file_path, max_read_buffer_size);
return &streams[stream_name]->compressed;
};
if (deserialize_states.count(name) == 0)
type.deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]);
type.deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]);
type.deserializeBinaryBulkWithMultipleStreams(column, limit, settings, deserialize_states[name]);
}
@ -429,6 +410,8 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const Storage
{
assert(table_path != new_path_to_table_data);
{
std::unique_lock<std::shared_mutex> lock(rwlock);
disk->moveDirectory(table_path, new_path_to_table_data);
table_path = new_path_to_table_data;
@ -441,16 +424,6 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const Storage
}
static std::chrono::seconds getLockTimeout(const Context & context)
{
const Settings & settings = context.getSettingsRef();
Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds();
if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout)
lock_timeout = settings.max_execution_time.totalSeconds();
return std::chrono::seconds{lock_timeout};
}
Pipe StorageTinyLog::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
@ -464,40 +437,28 @@ Pipe StorageTinyLog::read(
// When reading, we lock the entire storage, because we only have one file
// per column and can't modify it concurrently.
const Settings & settings = context.getSettingsRef();
std::shared_lock lock{rwlock, getLockTimeout(context)};
if (!lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
/// No need to hold lock while reading because we read fixed range of data that does not change while appending more data.
return Pipe(std::make_shared<TinyLogSource>(
max_block_size,
Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)),
*this,
settings.max_read_buffer_size,
file_checker.getFileSizes()));
max_block_size, Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)), *this, context.getSettingsRef().max_read_buffer_size));
}
BlockOutputStreamPtr StorageTinyLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context)
BlockOutputStreamPtr StorageTinyLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/)
{
return std::make_shared<TinyLogBlockOutputStream>(*this, metadata_snapshot, std::unique_lock{rwlock, getLockTimeout(context)});
return std::make_shared<TinyLogBlockOutputStream>(*this, metadata_snapshot);
}
CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context & context)
CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context & /* context */)
{
std::shared_lock lock(rwlock, getLockTimeout(context));
if (!lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
std::shared_lock<std::shared_mutex> lock(rwlock);
return file_checker.check();
}
void StorageTinyLog::truncate(
const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &)
{
std::unique_lock<std::shared_mutex> lock(rwlock);
disk->clearDirectory(table_path);
files.clear();
@ -507,6 +468,14 @@ void StorageTinyLog::truncate(
addFiles(column.name, *column.type);
}
void StorageTinyLog::drop()
{
std::unique_lock<std::shared_mutex> lock(rwlock);
if (disk->exists(table_path))
disk->removeRecursive(table_path);
files.clear();
}
void registerStorageTinyLog(StorageFactory & factory)
{

View File

@ -43,6 +43,8 @@ public:
void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override;
void drop() override;
protected:
StorageTinyLog(
DiskPtr disk_,
@ -68,7 +70,7 @@ private:
Files files;
FileChecker file_checker;
mutable std::shared_timed_mutex rwlock;
mutable std::shared_mutex rwlock;
Poco::Logger * log;

View File

@ -8,6 +8,7 @@
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/typeid_cast.h>
@ -42,7 +43,7 @@ NamesAndTypesList StorageSystemZooKeeper::getNamesAndTypes()
}
static bool extractPathImpl(const IAST & elem, String & res)
static bool extractPathImpl(const IAST & elem, String & res, const Context & context)
{
const auto * function = elem.as<ASTFunction>();
if (!function)
@ -51,7 +52,7 @@ static bool extractPathImpl(const IAST & elem, String & res)
if (function->name == "and")
{
for (const auto & child : function->arguments->children)
if (extractPathImpl(*child, res))
if (extractPathImpl(*child, res, context))
return true;
return false;
@ -60,23 +61,24 @@ static bool extractPathImpl(const IAST & elem, String & res)
if (function->name == "equals")
{
const auto & args = function->arguments->as<ASTExpressionList &>();
const IAST * value;
ASTPtr value;
if (args.children.size() != 2)
return false;
const ASTIdentifier * ident;
if ((ident = args.children.at(0)->as<ASTIdentifier>()))
value = args.children.at(1).get();
value = args.children.at(1);
else if ((ident = args.children.at(1)->as<ASTIdentifier>()))
value = args.children.at(0).get();
value = args.children.at(0);
else
return false;
if (ident->name != "path")
return false;
const auto * literal = value->as<ASTLiteral>();
auto evaluated = evaluateConstantExpressionAsLiteral(value, context);
const auto * literal = evaluated->as<ASTLiteral>();
if (!literal)
return false;
@ -93,20 +95,20 @@ static bool extractPathImpl(const IAST & elem, String & res)
/** Retrieve from the query a condition of the form `path = 'path'`, from conjunctions in the WHERE clause.
*/
static String extractPath(const ASTPtr & query)
static String extractPath(const ASTPtr & query, const Context & context)
{
const auto & select = query->as<ASTSelectQuery &>();
if (!select.where())
return "";
String res;
return extractPathImpl(*select.where(), res) ? res : "";
return extractPathImpl(*select.where(), res, context) ? res : "";
}
void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const
{
String path = extractPath(query_info.query);
String path = extractPath(query_info.query, context);
if (path.empty())
throw Exception("SELECT from system.zookeeper table must contain condition like path = 'path' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS);

View File

@ -1,6 +1,6 @@
#pragma once
#include <set>
#include <unordered_set>
#include <Core/Block.h>
#include <Parsers/IAST_fwd.h>
@ -30,9 +30,9 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & c
/// Extract from the input stream a set of `name` column values
template <typename T>
std::multiset<T> extractSingleValueFromBlock(const Block & block, const String & name)
auto extractSingleValueFromBlock(const Block & block, const String & name)
{
std::multiset<T> res;
std::unordered_set<T> res;
const ColumnWithTypeAndName & data = block.getByName(name);
size_t rows = block.rows();
for (size_t i = 0; i < rows; ++i)

View File

@ -237,7 +237,7 @@
"with_coverage": false
}
},
"Functional stateful tests (release, DatabaseAtomic)": {
"Functional stateful tests (release, DatabaseOrdinary)": {
"required_build_properties": {
"compiler": "gcc-10",
"package_type": "deb",
@ -345,7 +345,7 @@
"with_coverage": false
}
},
"Functional stateless tests (release, DatabaseAtomic)": {
"Functional stateless tests (release, DatabaseOrdinary)": {
"required_build_properties": {
"compiler": "gcc-10",
"package_type": "deb",
@ -441,6 +441,18 @@
"with_coverage": false
}
},
"Integration tests flaky check (asan)": {
"required_build_properties": {
"compiler": "clang-11",
"package_type": "deb",
"build_type": "relwithdebuginfo",
"sanitizer": "address",
"bundled": "bundled",
"splitted": "unsplitted",
"clang-tidy": "disable",
"with_coverage": false
}
},
"Compatibility check": {
"required_build_properties": {
"compiler": "gcc-10",

View File

@ -107,9 +107,9 @@ def remove_control_characters(s):
return s
def get_db_engine(args):
if args.atomic_db_engine:
return " ENGINE=Atomic"
return ""
if args.db_engine:
return " ENGINE=" + args.db_engine
return "" # Will use default engine
def run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file):
@ -303,6 +303,12 @@ def run_tests_array(all_tests_with_params):
clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
clickhouse_proc.communicate("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite))
if clickhouse_proc.returncode != 0:
failures += 1
print("Server does not respond to health check")
SERVER_DIED = True
break
reference_file = os.path.join(suite_dir, name) + '.reference'
stdout_file = os.path.join(suite_tmp_dir, name) + '.stdout'
stderr_file = os.path.join(suite_tmp_dir, name) + '.stderr'
@ -456,7 +462,7 @@ class BuildFlags(object):
DEBUG = 'debug-build'
UNBUNDLED = 'unbundled-build'
RELEASE = 'release-build'
DATABASE_ATOMIC = 'database-atomic'
DATABASE_ORDINARY = 'database-ordinary'
POLYMORPHIC_PARTS = 'polymorphic-parts'
@ -501,8 +507,8 @@ def collect_build_flags(client):
(stdout, stderr) = clickhouse_proc.communicate("SELECT value FROM system.settings WHERE name = 'default_database_engine'")
if clickhouse_proc.returncode == 0:
if 'Atomic' in stdout:
result.append(BuildFlags.DATABASE_ATOMIC)
if 'Ordinary' in stdout:
result.append(BuildFlags.DATABASE_ORDINARY)
else:
raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr))
@ -792,7 +798,7 @@ if __name__ == '__main__':
parser.add_argument('-r', '--server-check-retries', default=30, type=int, help='Num of tries to execute SELECT 1 before tests started')
parser.add_argument('--skip-list-path', help="Path to skip-list file")
parser.add_argument('--use-skip-list', action='store_true', default=False, help="Use skip list to skip tests if found")
parser.add_argument('--atomic-db-engine', action='store_true', help='Create databases with Atomic engine by default')
parser.add_argument('--db-engine', help='Database engine name')
parser.add_argument('--no-stateless', action='store_true', help='Disable all stateless tests')
parser.add_argument('--no-stateful', action='store_true', help='Disable all stateful tests')

8
tests/config/README.md Normal file
View File

@ -0,0 +1,8 @@
# ClickHouse configs for test environment
## How to use
CI use these configs in all checks installing them with `install.sh` script. If you want to run all tests from `tests/queries/0_stateless` and `test/queries/1_stateful` on your local machine you have to set up configs from this directory for your `clickhouse-server`. The most simple way is to install them using `install.sh` script. Other option is just copy files into your clickhouse config directory.
## How to add new config
Just place file `.xml` with new config into appropriate directory and add `ln` command into `install.sh` script. After that CI will use this config in all tests runs.

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