Merge branch 'master' into rename-const-context-ptr

This commit is contained in:
alexey-milovidov 2021-06-12 03:25:09 +03:00 committed by GitHub
commit 05d1af153c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
415 changed files with 7016 additions and 2621 deletions

5
.gitignore vendored
View File

@ -14,6 +14,11 @@
/build-*
/tests/venv
# logs
*.log
*.stderr
*.stdout
/docs/build
/docs/publish
/docs/edit

View File

@ -2,6 +2,8 @@
#### Upgrade Notes
* One bug has been found after release: [#25187](https://github.com/ClickHouse/ClickHouse/issues/25187).
* Do not upgrade if you have partition key with `UUID`.
* `zstd` compression library is updated to v1.5.0. You may get messages about "checksum does not match" in replication. These messages are expected due to update of compression algorithm and you can ignore them. These messages are informational and do not indicate any kinds of undesired behaviour.
* The setting `compile_expressions` is enabled by default. Although it has been heavily tested on variety of scenarios, if you find some undesired behaviour on your servers, you can try turning this setting off.
* Values of `UUID` type cannot be compared with integer. For example, instead of writing `uuid != 0` type `uuid != '00000000-0000-0000-0000-000000000000'`.
@ -763,6 +765,7 @@
* Allow using extended integer types (`Int128`, `Int256`, `UInt256`) in `avg` and `avgWeighted` functions. Also allow using different types (integer, decimal, floating point) for value and for weight in `avgWeighted` function. This is a backward-incompatible change: now the `avg` and `avgWeighted` functions always return `Float64` (as documented). Before this change the return type for `Decimal` arguments was also `Decimal`. [#15419](https://github.com/ClickHouse/ClickHouse/pull/15419) ([Mike](https://github.com/myrrc)).
* Expression `toUUID(N)` no longer works. Replace with `toUUID('00000000-0000-0000-0000-000000000000')`. This change is motivated by non-obvious results of `toUUID(N)` where N is non zero.
* SSL Certificates with incorrect "key usage" are rejected. In previous versions they are used to work. See [#19262](https://github.com/ClickHouse/ClickHouse/issues/19262).
* `incl` references to substitutions file (`/etc/metrika.xml`) were removed from the default config (`<remote_servers>`, `<zookeeper>`, `<macros>`, `<compression>`, `<networks>`). If you were using substitutions file and were relying on those implicit references, you should put them back manually and explicitly by adding corresponding sections with `incl="..."` attributes before the update. See [#18740](https://github.com/ClickHouse/ClickHouse/pull/18740) ([alexey-milovidov](https://github.com/alexey-milovidov)).
#### New Feature

View File

@ -4,12 +4,14 @@
#include <Core/Block.h>
#include <Interpreters/InternalTextLogsQueue.h>
#include <Interpreters/TextLog.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <sys/time.h>
#include <Poco/Message.h>
#include <Common/CurrentThread.h>
#include <Common/DNSResolver.h>
#include <common/getThreadId.h>
#include <Common/SensitiveDataMasker.h>
#include <Common/IO.h>
namespace DB
{
@ -26,16 +28,48 @@ void OwnSplitChannel::log(const Poco::Message & msg)
auto matches = masker->wipeSensitiveData(message_text);
if (matches > 0)
{
logSplit({msg, message_text}); // we will continue with the copy of original message with text modified
tryLogSplit({msg, message_text}); // we will continue with the copy of original message with text modified
return;
}
}
logSplit(msg);
tryLogSplit(msg);
}
void OwnSplitChannel::tryLogSplit(const Poco::Message & msg)
{
try
{
logSplit(msg);
}
/// It is better to catch the errors here in order to avoid
/// breaking some functionality because of unexpected "File not
/// found" (or similar) error.
///
/// For example StorageDistributedDirectoryMonitor will mark batch
/// as broken, some MergeTree code can also be affected.
///
/// Also note, that we cannot log the exception here, since this
/// will lead to recursion, using regular tryLogCurrentException().
/// but let's log it into the stderr at least.
catch (...)
{
MemoryTracker::LockExceptionInThread lock_memory_tracker(VariableContext::Global);
const std::string & exception_message = getCurrentExceptionMessage(true);
const std::string & message = msg.getText();
/// NOTE: errors are ignored, since nothing can be done.
writeRetry(STDERR_FILENO, "Cannot add message to the log: ");
writeRetry(STDERR_FILENO, message.data(), message.size());
writeRetry(STDERR_FILENO, "\n");
writeRetry(STDERR_FILENO, exception_message.data(), exception_message.size());
writeRetry(STDERR_FILENO, "\n");
}
}
void OwnSplitChannel::logSplit(const Poco::Message & msg)
{
ExtendedLogMessage msg_ext = ExtendedLogMessage::getFrom(msg);

View File

@ -24,6 +24,7 @@ public:
private:
void logSplit(const Poco::Message & msg);
void tryLogSplit(const Poco::Message & msg);
using ChannelPtr = Poco::AutoPtr<Poco::Channel>;
/// Handler and its pointer casted to extended interface

View File

@ -1,7 +1,7 @@
if(NOT OS_FREEBSD AND NOT APPLE)
if(NOT OS_FREEBSD)
option(ENABLE_S3 "Enable S3" ${ENABLE_LIBRARIES})
elseif(ENABLE_S3 OR USE_INTERNAL_AWS_S3_LIBRARY)
message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use S3 on Apple or FreeBSD")
message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use S3 on FreeBSD")
endif()
if(NOT ENABLE_S3)

2
contrib/avro vendored

@ -1 +1 @@
Subproject commit 1ee16d8c5a7808acff5cf0475f771195d9aa3faa
Subproject commit e43c46e87fd32eafdc09471e95344555454c5ef8

2
contrib/croaring vendored

@ -1 +1 @@
Subproject commit d8402939b5c9fc134fd4fcf058fe0f7006d2b129
Subproject commit 2c867e9f9c9e2a3a7032791f94c4c7ae3013f6e0

View File

@ -1,6 +1,6 @@
if (SANITIZE OR NOT (
((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_ARM OR ARCH_PPC64LE)) OR
(OS_DARWIN AND CMAKE_BUILD_TYPE STREQUAL "RelWithDebInfo")
(OS_DARWIN AND (CMAKE_BUILD_TYPE STREQUAL "RelWithDebInfo" OR CMAKE_BUILD_TYPE STREQUAL "Debug"))
))
if (ENABLE_JEMALLOC)
message (${RECONFIGURE_MESSAGE_LEVEL}

View File

@ -1 +1 @@
#*/10 * * * * root (which service > /dev/null 2>&1 && (service clickhouse-server condstart ||:)) || /etc/init.d/clickhouse-server condstart > /dev/null 2>&1
#*/10 * * * * root ((which service > /dev/null 2>&1 && (service clickhouse-server condstart ||:)) || /etc/init.d/clickhouse-server condstart) > /dev/null 2>&1

View File

@ -229,6 +229,7 @@ status()
case "$1" in
status)
status
exit 0
;;
esac

View File

@ -22,9 +22,9 @@ ENV SHA=nosha
ENV DATA="data"
CMD mkdir -p $BUILD_DIRECTORY && cd $BUILD_DIRECTORY && \
cmake $SOURCE_DIRECTORY -DCMAKE_CXX_COMPILER=/usr/bin/clang\+\+-11 -DCMAKE_C_COMPILER=/usr/bin/clang-11 -DCMAKE_EXPORT_COMPILE_COMMANDS=ON && \
cmake $SOURCE_DIRECTORY -DCMAKE_CXX_COMPILER=/usr/bin/clang\+\+-11 -DCMAKE_C_COMPILER=/usr/bin/clang-11 -DCMAKE_EXPORT_COMPILE_COMMANDS=ON -DENABLE_EMBEDDED_COMPILER=0 -DENABLE_S3=0 && \
mkdir -p $HTML_RESULT_DIRECTORY && \
$CODEGEN -b $BUILD_DIRECTORY -a -o $HTML_RESULT_DIRECTORY -p ClickHouse:$SOURCE_DIRECTORY:$SHA -d $DATA && \
$CODEGEN -b $BUILD_DIRECTORY -a -o $HTML_RESULT_DIRECTORY -p ClickHouse:$SOURCE_DIRECTORY:$SHA -d $DATA | ts '%Y-%m-%d %H:%M:%S' && \
cp -r $STATIC_DATA $HTML_RESULT_DIRECTORY/ &&\
$CODEINDEX $HTML_RESULT_DIRECTORY -d $DATA && \
$CODEINDEX $HTML_RESULT_DIRECTORY -d $DATA | ts '%Y-%m-%d %H:%M:%S' && \
mv $HTML_RESULT_DIRECTORY /test_output

View File

@ -97,14 +97,10 @@ function fuzz
NEW_TESTS_OPT="${NEW_TESTS_OPT:-}"
fi
export CLICKHOUSE_WATCHDOG_ENABLE=0 # interferes with gdb
clickhouse-server --config-file db/config.xml -- --path db 2>&1 | tail -100000 > server.log &
server_pid=$!
kill -0 $server_pid
while ! clickhouse-client --query "select 1" && kill -0 $server_pid ; do echo . ; sleep 1 ; done
clickhouse-client --query "select 1"
kill -0 $server_pid
echo Server started
echo "
handle all noprint
@ -115,12 +111,31 @@ thread apply all backtrace
continue
" > script.gdb
gdb -batch -command script.gdb -p "$(pidof clickhouse-server)" &
gdb -batch -command script.gdb -p $server_pid &
# Check connectivity after we attach gdb, because it might cause the server
# to freeze and the fuzzer will fail.
for _ in {1..60}
do
sleep 1
if clickhouse-client --query "select 1"
then
break
fi
done
clickhouse-client --query "select 1" # This checks that the server is responding
kill -0 $server_pid # This checks that it is our server that is started and not some other one
echo Server started and responded
# SC2012: Use find instead of ls to better handle non-alphanumeric filenames. They are all alphanumeric.
# SC2046: Quote this to prevent word splitting. Actually I need word splitting.
# shellcheck disable=SC2012,SC2046
clickhouse-client --query-fuzzer-runs=1000 --queries-file $(ls -1 ch/tests/queries/0_stateless/*.sql | sort -R) $NEW_TESTS_OPT \
clickhouse-client \
--receive_timeout=10 \
--receive_data_timeout_ms=10000 \
--query-fuzzer-runs=1000 \
--queries-file $(ls -1 ch/tests/queries/0_stateless/*.sql | sort -R) \
$NEW_TESTS_OPT \
> >(tail -n 100000 > fuzzer.log) \
2>&1 &
fuzzer_pid=$!
@ -198,13 +213,17 @@ continue
echo "success" > status.txt
echo "OK" > description.txt
else
# The server was alive, but the fuzzer returned some error. Probably this
# is a problem in the fuzzer itself. Don't grep the server log in this
# case, because we will find a message about normal server termination
# (Received signal 15), which is confusing.
# The server was alive, but the fuzzer returned some error. This might
# be some client-side error detected by fuzzing, or a problem in the
# fuzzer itself. Don't grep the server log in this case, because we will
# find a message about normal server termination (Received signal 15),
# which is confusing.
task_exit_code=$fuzzer_exit_code
echo "failure" > status.txt
echo "Fuzzer failed ($fuzzer_exit_code). See the logs." > description.txt
{ grep -o "Found error:.*" fuzzer.log \
|| grep -o "Exception.*" fuzzer.log \
|| echo "Fuzzer failed ($fuzzer_exit_code). See the logs." ; } \
| tail -1 > description.txt
fi
}

View File

@ -4,9 +4,9 @@ services:
image: sequenceiq/hadoop-docker:2.7.0
hostname: hdfs1
restart: always
ports:
- ${HDFS_NAME_EXTERNAL_PORT}:${HDFS_NAME_INTERNAL_PORT} #50070
- ${HDFS_DATA_EXTERNAL_PORT}:${HDFS_DATA_INTERNAL_PORT} #50075
expose:
- ${HDFS_NAME_PORT}
- ${HDFS_DATA_PORT}
entrypoint: /etc/bootstrap.sh -d
volumes:
- type: ${HDFS_FS:-tmpfs}

View File

@ -0,0 +1,23 @@
version: '2.3'
services:
bridge1:
image: yandex/clickhouse-jdbc-bridge
command: |
/bin/bash -c 'cat << EOF > config/datasources/self.json
{
"self": {
"jdbcUrl": "jdbc:clickhouse://instance:8123/test",
"username": "default",
"password": "",
"maximumPoolSize": 5
}
}
EOF
./docker-entrypoint.sh'
ports:
- 9020:9019
healthcheck:
test: ["CMD", "curl", "-s", "localhost:9019/ping"]
interval: 5s
timeout: 3s
retries: 30

View File

@ -14,9 +14,9 @@ services:
- type: ${KERBERIZED_HDFS_FS:-tmpfs}
source: ${KERBERIZED_HDFS_LOGS:-}
target: /var/log/hadoop-hdfs
ports:
- ${KERBERIZED_HDFS_NAME_EXTERNAL_PORT}:${KERBERIZED_HDFS_NAME_INTERNAL_PORT} #50070
- ${KERBERIZED_HDFS_DATA_EXTERNAL_PORT}:${KERBERIZED_HDFS_DATA_INTERNAL_PORT} #1006
expose:
- ${KERBERIZED_HDFS_NAME_PORT}
- ${KERBERIZED_HDFS_DATA_PORT}
depends_on:
- hdfskerberos
entrypoint: /etc/bootstrap.sh -d
@ -28,4 +28,4 @@ services:
- ${KERBERIZED_HDFS_DIR}/secrets:/tmp/keytab
- ${KERBERIZED_HDFS_DIR}/../../kerberos_image_config.sh:/config.sh
- /dev/urandom:/dev/random
ports: [88, 749]
expose: [88, 749]

View File

@ -10,7 +10,7 @@ echo '{
"storage-driver": "overlay2",
"insecure-registries" : ["dockerhub-proxy.sas.yp-c.yandex.net:5000"],
"registry-mirrors" : ["http://dockerhub-proxy.sas.yp-c.yandex.net:5000"]
}' | dd of=/etc/docker/daemon.json
}' | dd of=/etc/docker/daemon.json 2>/dev/null
dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log &

View File

@ -554,12 +554,6 @@ create table query_metric_stats_denorm engine File(TSVWithNamesAndTypes,
" 2> >(tee -a analyze/errors.log 1>&2)
# Fetch historical query variability thresholds from the CI database
clickhouse-local --query "
left join file('analyze/report-thresholds.tsv', TSV,
'test text, report_threshold float') thresholds
on query_metric_stats.test = thresholds.test
"
if [ -v CHPC_DATABASE_URL ]
then
set +x # Don't show password in the log
@ -577,7 +571,8 @@ then
--date_time_input_format=best_effort)
# Precision is going to be 1.5 times worse for PRs. How do I know it? I ran this:
# Precision is going to be 1.5 times worse for PRs, because we run the queries
# less times. How do I know it? I ran this:
# SELECT quantilesExact(0., 0.1, 0.5, 0.75, 0.95, 1.)(p / m)
# FROM
# (
@ -592,19 +587,27 @@ then
# query_display_name
# HAVING count(*) > 100
# )
# The file can be empty if the server is inaccessible, so we can't use TSVWithNamesAndTypes.
#
# The file can be empty if the server is inaccessible, so we can't use
# TSVWithNamesAndTypes.
#
"${client[@]}" --query "
select test, query_index,
quantileExact(0.99)(abs(diff)) max_diff,
quantileExactIf(0.99)(stat_threshold, abs(diff) < stat_threshold) * 1.5 max_stat_threshold,
quantileExact(0.99)(abs(diff)) * 1.5 AS max_diff,
quantileExactIf(0.99)(stat_threshold, abs(diff) < stat_threshold) * 1.5 AS max_stat_threshold,
query_display_name
from query_metrics_v2
where event_date > now() - interval 1 month
-- We use results at least one week in the past, so that the current
-- changes do not immediately influence the statistics, and we have
-- some time to notice that something is wrong.
where event_date between now() - interval 1 month - interval 1 week
and now() - interval 1 week
and metric = 'client_time'
and pr_number = 0
group by test, query_index, query_display_name
having count(*) > 100
" > analyze/historical-thresholds.tsv
set -x
else
touch analyze/historical-thresholds.tsv
fi
@ -1224,6 +1227,55 @@ unset IFS
function upload_results
{
# Prepare info for the CI checks table.
rm ci-checks.tsv
clickhouse-local --query "
create view queries as select * from file('report/queries.tsv', TSVWithNamesAndTypes,
'changed_fail int, changed_show int, unstable_fail int, unstable_show int,
left float, right float, diff float, stat_threshold float,
test text, query_index int, query_display_name text');
create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv')
as select
$PR_TO_TEST pull_request_number,
'$SHA_TO_TEST' commit_sha,
'Performance' check_name,
'$(sed -n 's/.*<!--status: \(.*\)-->/\1/p' report.html)' check_status,
-- TODO toDateTime() can't parse output of 'date', so no time for now.
($(date +%s) - $CHPC_CHECK_START_TIMESTAMP) * 1000 check_duration_ms,
fromUnixTimestamp($CHPC_CHECK_START_TIMESTAMP) check_start_time,
test_name,
test_status,
test_duration_ms,
report_url,
$PR_TO_TEST = 0
? 'https://github.com/ClickHouse/ClickHouse/commit/$SHA_TO_TEST'
: 'https://github.com/ClickHouse/ClickHouse/pull/$PR_TO_TEST' pull_request_url,
'' commit_url,
'' task_url,
'' base_ref,
'' base_repo,
'' head_ref,
'' head_repo
from (
select '' test_name,
'$(sed -n 's/.*<!--message: \(.*\)-->/\1/p' report.html)' test_status,
0 test_duration_ms,
'https://clickhouse-test-reports.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/performance_comparison/report.html#fail1' report_url
union all
select test || ' #' || toString(query_index), 'slower' test_status, 0 test_duration_ms,
'https://clickhouse-test-reports.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/performance_comparison/report.html#changes-in-performance.'
|| test || '.' || toString(query_index) report_url
from queries where changed_fail != 0 and diff > 0
union all
select test || ' #' || toString(query_index), 'unstable' test_status, 0 test_duration_ms,
'https://clickhouse-test-reports.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/performance_comparison/report.html#unstable-queries.'
|| test || '.' || toString(query_index) report_url
from queries where unstable_fail != 0
)
;
"
if ! [ -v CHPC_DATABASE_URL ]
then
echo Database for test results is not specified, will not upload them.
@ -1292,6 +1344,10 @@ $REF_SHA $SHA_TO_TEST $(numactl --show | sed -n 's/^cpubind:[[:space:]]\+/numact
$REF_SHA $SHA_TO_TEST $(numactl --hardware | sed -n 's/^available:[[:space:]]\+/numactl-available /p')
EOF
# Also insert some data about the check into the CI checks table.
"${client[@]}" --query "INSERT INTO "'"'"gh-data"'"'".checks FORMAT TSVWithNamesAndTypes" \
< ci-checks.tsv
set -x
}

View File

@ -1,6 +1,9 @@
#!/bin/bash
set -ex
CHPC_CHECK_START_TIMESTAMP="$(date +%s)"
export CHPC_CHECK_START_TIMESTAMP
# Use the packaged repository to find the revision we will compare to.
function find_reference_sha
{

View File

@ -561,8 +561,9 @@ if args.report == 'main':
# Don't show mildly unstable queries, only the very unstable ones we
# treat as errors.
if very_unstable_queries:
error_tests += very_unstable_queries
status = 'failure'
if very_unstable_queries > 3:
error_tests += very_unstable_queries
status = 'failure'
message_array.append(str(very_unstable_queries) + ' unstable')
error_tests += slow_average_tests

View File

@ -8,6 +8,7 @@ RUN apt-get update -y && \
python3-wheel \
brotli \
netcat-openbsd \
postgresql-client \
zstd
RUN python3 -m pip install \

View File

@ -34,6 +34,16 @@ def get_options(i):
if i % 2 == 1:
options.append(" --database=test_{}".format(i))
if i % 7 == 0:
options.append(" --client-option='join_use_nulls=1'")
if i % 14 == 0:
options.append(' --client-option="join_algorithm=\'partial_merge\'"')
if i % 21 == 0:
options.append(' --client-option="join_algorithm=\'auto\'"')
options.append(' --client-option="max_rows_in_join=1000"')
if i == 13:
options.append(" --client-option='memory_tracker_fault_probability=0.00001'")

View File

@ -1,6 +1,15 @@
#!/bin/bash
set -e
echo "Configure to use Yandex dockerhub-proxy"
mkdir -p /etc/docker/
cat > /etc/docker/daemon.json << EOF
{
"insecure-registries" : ["dockerhub-proxy.sas.yp-c.yandex.net:5000"],
"registry-mirrors" : ["http://dockerhub-proxy.sas.yp-c.yandex.net:5000"]
}
EOF
dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 &>/var/log/somefile &
set +e
@ -16,14 +25,6 @@ while true; do
done
set -e
echo "Configure to use Yandex dockerhub-proxy"
cat > /etc/docker/daemon.json << EOF
{
"insecure-registries": ["dockerhub-proxy.sas.yp-c.yandex.net:5000"],
"registry-mirrors": ["dockerhub-proxy.sas.yp-c.yandex.net:5000"]
}
EOF
echo "Start tests"
export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/clickhouse
export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse

View File

@ -7,7 +7,7 @@ toc_title: JDBC
Allows ClickHouse to connect to external databases via [JDBC](https://en.wikipedia.org/wiki/Java_Database_Connectivity).
To implement the JDBC connection, ClickHouse uses the separate program [clickhouse-jdbc-bridge](https://github.com/alex-krash/clickhouse-jdbc-bridge) that should run as a daemon.
To implement the JDBC connection, ClickHouse uses the separate program [clickhouse-jdbc-bridge](https://github.com/ClickHouse/clickhouse-jdbc-bridge) that should run as a daemon.
This engine supports the [Nullable](../../../sql-reference/data-types/nullable.md) data type.
@ -18,19 +18,20 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name
(
columns list...
)
ENGINE = JDBC(dbms_uri, external_database, external_table)
ENGINE = JDBC(datasource_uri, external_database, external_table)
```
**Engine Parameters**
- `dbms_uri` — URI of an external DBMS.
Format: `jdbc:<driver_name>://<host_name>:<port>/?user=<username>&password=<password>`.
- `datasource_uri` — URI or name of an external DBMS.
URI Format: `jdbc:<driver_name>://<host_name>:<port>/?user=<username>&password=<password>`.
Example for MySQL: `jdbc:mysql://localhost:3306/?user=root&password=root`.
- `external_database` — Database in an external DBMS.
- `external_table` — Name of the table in `external_database`.
- `external_table` — Name of the table in `external_database` or a select query like `select * from table1 where column1=1`.
## Usage Example {#usage-example}
@ -81,6 +82,12 @@ FROM jdbc_table
└────────┴──────────────┴───────┴────────────────┘
```
``` sql
INSERT INTO jdbc_table(`int_id`, `float`)
SELECT toInt32(number), toFloat32(number * 1.0)
FROM system.numbers
```
## See Also {#see-also}
- [JDBC table function](../../../sql-reference/table-functions/jdbc.md).

View File

@ -32,7 +32,7 @@ CREATE TABLE `ontime`
`Reporting_Airline` String,
`DOT_ID_Reporting_Airline` Int32,
`IATA_CODE_Reporting_Airline` String,
`Tail_Number` Int32,
`Tail_Number` String,
`Flight_Number_Reporting_Airline` String,
`OriginAirportID` Int32,
`OriginAirportSeqID` Int32,

View File

@ -94,6 +94,15 @@ For production environments, its recommended to use the latest `stable`-versi
To run ClickHouse inside Docker follow the guide on [Docker Hub](https://hub.docker.com/r/yandex/clickhouse-server/). Those images use official `deb` packages inside.
### Single Binary
You can install ClickHouse on Linux using single portable binary from the latest commit of the `master` branch: [https://builds.clickhouse.tech/master/amd64/clickhouse].
```
curl -O 'https://builds.clickhouse.tech/master/amd64/clickhouse' && chmod a+x clickhouse
sudo ./clickhouse install
```
### From Precompiled Binaries for Non-Standard Environments {#from-binaries-non-linux}
For non-Linux operating systems and for AArch64 CPU arhitecture, ClickHouse builds are provided as a cross-compiled binary from the latest commit of the `master` branch (with a few hours delay).
@ -104,7 +113,7 @@ For non-Linux operating systems and for AArch64 CPU arhitecture, ClickHouse buil
After downloading, you can use the `clickhouse client` to connect to the server, or `clickhouse local` to process local data.
Run `sudo ./clickhouse install` if you want to install clickhouse system-wide (also with needed configuration files, configuring users etc.). After that run `clickhouse start` commands to start the clickhouse-server and `clickhouse-client` to connect to it.
Run `sudo ./clickhouse install` if you want to install clickhouse system-wide (also with needed configuration files, configuring users etc.). After that run `clickhouse start` commands to start the clickhouse-server and `clickhouse-client` to connect to it.
These builds are not recommended for use in production environments because they are less thoroughly tested, but you can do so on your own risk. They also have only a subset of ClickHouse features available.

View File

@ -56,13 +56,13 @@ Note, that you can define multiple LDAP servers inside the `ldap_servers` sectio
- `port` — LDAP server port, default is `636` if `enable_tls` is set to `true`, `389` otherwise.
- `bind_dn` — Template used to construct the DN to bind to.
- The resulting DN will be constructed by replacing all `{user_name}` substrings of the template with the actual user name during each authentication attempt.
- `user_dn_detection` - Section with LDAP search parameters for detecting the actual user DN of the bound user.
- `user_dn_detection` Section with LDAP search parameters for detecting the actual user DN of the bound user.
- This is mainly used in search filters for further role mapping when the server is Active Directory. The resulting user DN will be used when replacing `{user_dn}` substrings wherever they are allowed. By default, user DN is set equal to bind DN, but once search is performed, it will be updated with to the actual detected user DN value.
- `base_dn` - Template used to construct the base DN for the LDAP search.
- `base_dn` Template used to construct the base DN for the LDAP search.
- The resulting DN will be constructed by replacing all `{user_name}` and `{bind_dn}` substrings of the template with the actual user name and bind DN during the LDAP search.
- `scope` - Scope of the LDAP search.
- `scope` Scope of the LDAP search.
- Accepted values are: `base`, `one_level`, `children`, `subtree` (the default).
- `search_filter` - Template used to construct the search filter for the LDAP search.
- `search_filter` Template used to construct the search filter for the LDAP search.
- The resulting filter will be constructed by replacing all `{user_name}`, `{bind_dn}`, and `{base_dn}` substrings of the template with the actual user name, bind DN, and base DN during the LDAP search.
- Note, that the special characters must be escaped properly in XML.
- `verification_cooldown` — A period of time, in seconds, after a successful bind attempt, during which the user will be assumed to be successfully authenticated for all consecutive requests without contacting the LDAP server.
@ -108,7 +108,6 @@ Note, that user `my_user` refers to `my_ldap_server`. This LDAP server must be c
When SQL-driven [Access Control and Account Management](../access-rights.md#access-control) is enabled, users that are authenticated by LDAP servers can also be created using the [CREATE USER](../../sql-reference/statements/create/user.md#create-user-statement) statement.
Query:
```sql

View File

@ -0,0 +1,107 @@
---
toc_priority: 146
toc_title: intervalLengthSum
---
# intervalLengthSum {#agg_function-intervallengthsum}
Calculates the total length of union of all ranges (segments on numeric axis).
**Syntax**
``` sql
intervalLengthSum(start, end)
```
**Arguments**
- `start` — The starting value of the interval. [Int32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Int64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Float32](../../../sql-reference/data-types/float.md#float32-float64), [Float64](../../../sql-reference/data-types/float.md#float32-float64), [DateTime](../../../sql-reference/data-types/datetime.md#data_type-datetime) or [Date](../../../sql-reference/data-types/date.md#data_type-date).
- `end` — The ending value of the interval. [Int32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Int64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Float32](../../../sql-reference/data-types/float.md#float32-float64), [Float64](../../../sql-reference/data-types/float.md#float32-float64), [DateTime](../../../sql-reference/data-types/datetime.md#data_type-datetime) or [Date](../../../sql-reference/data-types/date.md#data_type-date).
!!! info "Note"
Arguments must be of the same data type. Otherwise, an exception will be thrown.
**Returned value**
- Total length of union of all ranges (segments on numeric axis). Depending on the type of the argument, the return value may be [UInt64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64) or [Float64](../../../sql-reference/data-types/float.md#float32-float64) type.
**Examples**
1. Input table:
``` text
┌─id─┬─start─┬─end─┐
│ a │ 1.1 │ 2.9 │
│ a │ 2.5 │ 3.2 │
│ a │ 4 │ 5 │
└────┴───────┴─────┘
```
In this example, the arguments of the Float32 type are used. The function returns a value of the Float64 type.
Result is the sum of lengths of intervals `[1.1, 3.2]` (union of `[1.1, 2.9]` and `[2.5, 3.2]`) and `[4, 5]`
Query:
``` sql
SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM fl_interval GROUP BY id ORDER BY id;
```
Result:
``` text
┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐
│ a │ 3.1 │ Float64 │
└────┴───────────────────────────────┴───────────────────────────────────────────┘
```
2. Input table:
``` text
┌─id─┬───────────────start─┬─────────────────end─┐
│ a │ 2020-01-01 01:12:30 │ 2020-01-01 02:10:10 │
│ a │ 2020-01-01 02:05:30 │ 2020-01-01 02:50:31 │
│ a │ 2020-01-01 03:11:22 │ 2020-01-01 03:23:31 │
└────┴─────────────────────┴─────────────────────┘
```
In this example, the arguments of the DateTime type are used. The function returns a value in seconds.
Query:
``` sql
SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM dt_interval GROUP BY id ORDER BY id;
```
Result:
``` text
┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐
│ a │ 6610 │ UInt64 │
└────┴───────────────────────────────┴───────────────────────────────────────────┘
```
3. Input table:
``` text
┌─id─┬──────start─┬────────end─┐
│ a │ 2020-01-01 │ 2020-01-04 │
│ a │ 2020-01-12 │ 2020-01-18 │
└────┴────────────┴────────────┘
```
In this example, the arguments of the Date type are used. The function returns a value in days.
Query:
``` sql
SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM date_interval GROUP BY id ORDER BY id;
```
Result:
``` text
┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐
│ a │ 9 │ UInt64 │
└────┴───────────────────────────────┴───────────────────────────────────────────┘
```

View File

@ -4,7 +4,7 @@ toc_priority: 144
# sumCount {#agg_function-sumCount}
Calculates the sum of the numbers and counts the number of rows at the same time.
Calculates the sum of the numbers and counts the number of rows at the same time. The function is used by ClickHouse query optimizer: if there are multiple `sum`, `count` or `avg` functions in a query, they can be replaced to single `sumCount` function to reuse the calculations. The function is rarely needed to use explicitly.
**Syntax**

View File

@ -0,0 +1,40 @@
---
toc_priority: 145
---
# sumKahan {#agg_function-sumKahan}
Calculates the sum of the numbers with [Kahan compensated summation algorithm](https://en.wikipedia.org/wiki/Kahan_summation_algorithm)
Slower than [sum](./sum.md) function.
The compensation works only for [Float](../../../sql-reference/data-types/float.md) types.
**Syntax**
``` sql
sumKahan(x)
```
**Arguments**
- `x` — Input value, must be [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md), or [Decimal](../../../sql-reference/data-types/decimal.md).
**Returned value**
- the sum of numbers, with type [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md), or [Decimal](../../../sql-reference/data-types/decimal.md) depends on type of input arguments
**Example**
Query:
``` sql
SELECT sum(0.1), sumKahan(0.1) FROM numbers(10);
```
Result:
``` text
┌───────────sum(0.1)─┬─sumKahan(0.1)─┐
│ 0.9999999999999999 │ 1 │
└────────────────────┴───────────────┘
```

View File

@ -4,7 +4,7 @@ toc_priority: 109
# topKWeighted {#topkweighted}
Similar to `topK` but takes one additional argument of integer type - `weight`. Every value is accounted `weight` times for frequency calculation.
Returns an array of the approximately most frequent values in the specified column. The resulting array is sorted in descending order of approximate frequency of values (not by the values themselves). Additionally, the weight of the value is taken into account.
**Syntax**
@ -15,11 +15,8 @@ topKWeighted(N)(x, weight)
**Arguments**
- `N` — The number of elements to return.
**Arguments**
- `x` — The value.
- `weight` — The weight. [UInt8](../../../sql-reference/data-types/int-uint.md).
- `weight` — The weight. Every value is accounted `weight` times for frequency calculation. [UInt64](../../../sql-reference/data-types/int-uint.md).
**Returned value**
@ -40,3 +37,7 @@ Result:
│ [999,998,997,996,995,994,993,992,991,990] │
└───────────────────────────────────────────┘
```
**See Also**
- [topK](../../../sql-reference/aggregate-functions/reference/topk.md)

View File

@ -159,7 +159,7 @@ Configuration fields:
| Tag | Description | Required |
|------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------|
| `name` | Column name. | Yes |
| `type` | ClickHouse data type.<br/>ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.<br/>[Nullable](../../../sql-reference/data-types/nullable.md) is currently supported for [Flat](external-dicts-dict-layout.md#flat), [Hashed](external-dicts-dict-layout.md#dicts-external_dicts_dict_layout-hashed), [ComplexKeyHashed](external-dicts-dict-layout.md#complex-key-hashed), [Direct](external-dicts-dict-layout.md#direct), [ComplexKeyDirect](external-dicts-dict-layout.md#complex-key-direct), [RangeHashed](external-dicts-dict-layout.md#range-hashed), [Polygon](external-dicts-dict-polygon.md), [Cache](external-dicts-dict-layout.md#cache), [ComplexKeyCache](external-dicts-dict-layout.md#complex-key-cache), [SSDCache](external-dicts-dict-layout.md#ssd-cache), [SSDComplexKeyCache](external-dicts-dict-layout.md#complex-key-ssd-cache) dictionaries. In [IPTrie](external-dicts-dict-layout.md#ip-trie) dictionaries `Nullable` types are not supported. | Yes |
| `type` | ClickHouse data type: [UInt8](../../../sql-reference/data-types/int-uint.md), [UInt16](../../../sql-reference/data-types/int-uint.md), [UInt32](../../../sql-reference/data-types/int-uint.md), [UInt64](../../../sql-reference/data-types/int-uint.md), [Int8](../../../sql-reference/data-types/int-uint.md), [Int16](../../../sql-reference/data-types/int-uint.md), [Int32](../../../sql-reference/data-types/int-uint.md), [Int64](../../../sql-reference/data-types/int-uint.md), [Float32](../../../sql-reference/data-types/float.md), [Float64](../../../sql-reference/data-types/float.md), [UUID](../../../sql-reference/data-types/uuid.md), [Decimal32](../../../sql-reference/data-types/decimal.md), [Decimal64](../../../sql-reference/data-types/decimal.md), [Decimal128](../../../sql-reference/data-types/decimal.md), [Decimal256](../../../sql-reference/data-types/decimal.md), [String](../../../sql-reference/data-types/string.md).<br/>ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.<br/>[Nullable](../../../sql-reference/data-types/nullable.md) is currently supported for [Flat](external-dicts-dict-layout.md#flat), [Hashed](external-dicts-dict-layout.md#dicts-external_dicts_dict_layout-hashed), [ComplexKeyHashed](external-dicts-dict-layout.md#complex-key-hashed), [Direct](external-dicts-dict-layout.md#direct), [ComplexKeyDirect](external-dicts-dict-layout.md#complex-key-direct), [RangeHashed](external-dicts-dict-layout.md#range-hashed), [Polygon](external-dicts-dict-polygon.md), [Cache](external-dicts-dict-layout.md#cache), [ComplexKeyCache](external-dicts-dict-layout.md#complex-key-cache), [SSDCache](external-dicts-dict-layout.md#ssd-cache), [SSDComplexKeyCache](external-dicts-dict-layout.md#complex-key-ssd-cache) dictionaries. In [IPTrie](external-dicts-dict-layout.md#ip-trie) dictionaries `Nullable` types are not supported. | Yes |
| `null_value` | Default value for a non-existing element.<br/>In the example, it is an empty string. [NULL](../../syntax.md#null-literal) value can be used only for the `Nullable` types (see the previous line with types description). | Yes |
| `expression` | [Expression](../../../sql-reference/syntax.md#syntax-expressions) that ClickHouse executes on the value.<br/>The expression can be a column name in the remote SQL database. Thus, you can use it to create an alias for the remote column.<br/><br/>Default value: no expression. | No |
| <a name="hierarchical-dict-attr"></a> `hierarchical` | If `true`, the attribute contains the value of a parent key for the current key. See [Hierarchical Dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md).<br/><br/>Default value: `false`. | No |

View File

@ -265,7 +265,7 @@ Result:
```
!!! attention "Attention"
The return type `toStartOf*` functions described below is `Date` or `DateTime`. Though these functions can take `DateTime64` as an argument, passing them a `DateTime64` that is out of normal range (years 1970 - 2105) will give incorrect result.
The return type `toStartOf*` functions described below is `Date` or `DateTime`. Though these functions can take `DateTime64` as an argument, passing them a `DateTime64` that is out of the normal range (years 1925 - 2283) will give an incorrect result.
## toStartOfYear {#tostartofyear}

View File

@ -6,7 +6,7 @@ toc_title: RENAME
# RENAME Statement {#misc_operations-rename}
## RENAME DATABASE {#misc_operations-rename_database}
Renames database, support only for Atomic database engine
Renames database, it is supported only for Atomic database engine.
```
RENAME DATABASE atomic_database1 TO atomic_database2 [ON CLUSTER cluster]

View File

@ -79,7 +79,7 @@ Completely reloads a CatBoost model `model_name` if the configuration was update
**Syntax**
```sql
SYSTEM RELOAD MODEL [db.]<model_name>
SYSTEM RELOAD MODEL <model_name>
```
## DROP DNS CACHE {#query_language-system-drop-dns-cache}

View File

@ -5,9 +5,9 @@ toc_title: jdbc
# jdbc {#table-function-jdbc}
`jdbc(jdbc_connection_uri, schema, table)` - returns table that is connected via JDBC driver.
`jdbc(datasource, schema, table)` - returns table that is connected via JDBC driver.
This table function requires separate `clickhouse-jdbc-bridge` program to be running.
This table function requires separate [clickhouse-jdbc-bridge](https://github.com/ClickHouse/clickhouse-jdbc-bridge) program to be running.
It supports Nullable types (based on DDL of remote table that is queried).
**Examples**
@ -17,10 +17,22 @@ SELECT * FROM jdbc('jdbc:mysql://localhost:3306/?user=root&password=root', 'sche
```
``` sql
SELECT * FROM jdbc('mysql://localhost:3306/?user=root&password=root', 'schema', 'table')
SELECT * FROM jdbc('mysql://localhost:3306/?user=root&password=root', 'select * from schema.table')
```
``` sql
SELECT * FROM jdbc('datasource://mysql-local', 'schema', 'table')
SELECT * FROM jdbc('mysql-dev?p1=233', 'num Int32', 'select toInt32OrZero(''{{p1}}'') as num')
```
``` sql
SELECT *
FROM jdbc('mysql-dev?p1=233', 'num Int32', 'select toInt32OrZero(''{{p1}}'') as num')
```
``` sql
SELECT a.datasource AS server1, b.datasource AS server2, b.name AS db
FROM jdbc('mysql-dev?datasource_column', 'show databases') a
INNER JOIN jdbc('self?datasource_column', 'show databases') b ON a.Database = b.name
```
[Original article](https://clickhouse.tech/docs/en/query_language/table_functions/jdbc/) <!--hide-->

View File

@ -114,14 +114,14 @@ FROM s3('https://storage.yandexcloud.net/my-test-bucket-768/big_prefix/file-{000
Insert data into file `test-data.csv.gz`:
``` sql
INSERT INTO s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip')
INSERT INTO FUNCTION s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip')
VALUES ('test-data', 1), ('test-data-2', 2);
```
Insert data into file `test-data.csv.gz` from existing table:
``` sql
INSERT INTO s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip')
INSERT INTO FUNCTION s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip')
SELECT name, value FROM existing_table;
```

View File

@ -6,9 +6,7 @@ toc_title: Window Functions
# [experimental] Window Functions
!!! warning "Warning"
This is an experimental feature that is currently in development and is not ready
for general use. It will change in unpredictable backwards-incompatible ways in
the future releases. Set `allow_experimental_window_functions = 1` to enable it.
This is an experimental feature that is currently in development and is not ready for general use. It will change in unpredictable backwards-incompatible ways in the future releases. Set `allow_experimental_window_functions = 1` to enable it.
ClickHouse supports the standard grammar for defining windows and window functions. The following features are currently supported:

View File

@ -9,7 +9,7 @@ toc_title: JDBC
ClickHouseが外部データベースに接続できるようにする [JDBC](https://en.wikipedia.org/wiki/Java_Database_Connectivity).
JDBC接続を実装するには、ClickHouseは別のプログラムを使用します [clickhouse-jdbc-bridge](https://github.com/alex-krash/clickhouse-jdbc-bridge) うにしてくれました。
JDBC接続を実装するには、ClickHouseは別のプログラムを使用します [clickhouse-jdbc-bridge](https://github.com/ClickHouse/clickhouse-jdbc-bridge) うにしてくれました。
このエンジンは [Null可能](../../../sql-reference/data-types/nullable.md) データ型。
@ -20,19 +20,19 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name
(
columns list...
)
ENGINE = JDBC(dbms_uri, external_database, external_table)
ENGINE = JDBC(datasource_uri, external_database, external_table)
```
**エンジン変数**
- `dbms_uri` — URI of an external DBMS.
- `datasource_uri` — URI or name of an external DBMS.
形式: `jdbc:<driver_name>://<host_name>:<port>/?user=<username>&password=<password>`.
URI形式: `jdbc:<driver_name>://<host_name>:<port>/?user=<username>&password=<password>`.
MySQLの例: `jdbc:mysql://localhost:3306/?user=root&password=root`.
- `external_database` — Database in an external DBMS.
- `external_table` — Name of the table in `external_database`.
- `external_table` — Name of the table in `external_database` or a select query like `select * from table1 where column1=1`.
## 使用例 {#usage-example}
@ -83,6 +83,12 @@ FROM jdbc_table
└────────┴──────────────┴───────┴────────────────┘
```
``` sql
INSERT INTO jdbc_table(`int_id`, `float`)
SELECT toInt32(number), toFloat32(number * 1.0)
FROM system.numbers
```
## も参照。 {#see-also}
- [JDBCテーブル関数](../../../sql-reference/table-functions/jdbc.md).

View File

@ -7,9 +7,9 @@ toc_title: jdbc
# jdbc {#table-function-jdbc}
`jdbc(jdbc_connection_uri, schema, table)` -JDBCドライバ経由で接続されたテーブルを返します。
`jdbc(datasource, schema, table)` -JDBCドライバ経由で接続されたテーブルを返します。
このテーブル関数には、別々の `clickhouse-jdbc-bridge` 実行するプログラム。
このテーブル関数には、別々の [clickhouse-jdbc-bridge](https://github.com/ClickHouse/clickhouse-jdbc-bridge) 実行するプログラム。
Null許容型をサポートします(照会されるリモートテーブルのDDLに基づきます)。
**例**
@ -19,11 +19,22 @@ SELECT * FROM jdbc('jdbc:mysql://localhost:3306/?user=root&password=root', 'sche
```
``` sql
SELECT * FROM jdbc('mysql://localhost:3306/?user=root&password=root', 'schema', 'table')
SELECT * FROM jdbc('mysql://localhost:3306/?user=root&password=root', 'select * from schema.table')
```
``` sql
SELECT * FROM jdbc('datasource://mysql-local', 'schema', 'table')
SELECT * FROM jdbc('mysql-dev?p1=233', 'num Int32', 'select toInt32OrZero(''{{p1}}'') as num')
```
``` sql
SELECT *
FROM jdbc('mysql-dev?p1=233', 'num Int32', 'select toInt32OrZero(''{{p1}}'') as num')
```
``` sql
SELECT a.datasource AS server1, b.datasource AS server2, b.name AS db
FROM jdbc('mysql-dev?datasource_column', 'show databases') a
INNER JOIN jdbc('self?datasource_column', 'show databases') b ON a.Database = b.name
```
[元の記事](https://clickhouse.tech/docs/en/query_language/table_functions/jdbc/) <!--hide-->

View File

@ -7,7 +7,7 @@ toc_title: JDBC
Позволяет ClickHouse подключаться к внешним базам данных с помощью [JDBC](https://en.wikipedia.org/wiki/Java_Database_Connectivity).
Для реализации соединения по JDBC ClickHouse использует отдельную программу [clickhouse-jdbc-bridge](https://github.com/alex-krash/clickhouse-jdbc-bridge), которая должна запускаться как демон.
Для реализации соединения по JDBC ClickHouse использует отдельную программу [clickhouse-jdbc-bridge](https://github.com/ClickHouse/clickhouse-jdbc-bridge), которая должна запускаться как демон.
Движок поддерживает тип данных [Nullable](../../../engines/table-engines/integrations/jdbc.md).
@ -15,20 +15,20 @@ toc_title: JDBC
``` sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name
ENGINE = JDBC(dbms_uri, external_database, external_table)
ENGINE = JDBC(datasource_uri, external_database, external_table)
```
**Параметры движка**
- `dbms_uri` — URI внешней СУБД.
- `datasource_uri` — URI или имя внешней СУБД.
Формат: `jdbc:<driver_name>://<host_name>:<port>/?user=<username>&password=<password>`.
URI Формат: `jdbc:<driver_name>://<host_name>:<port>/?user=<username>&password=<password>`.
Пример для MySQL: `jdbc:mysql://localhost:3306/?user=root&password=root`.
- `external_database` — база данных во внешней СУБД.
- `external_table` — таблица в `external_database`.
- `external_table` — таблицы в `external_database` или запросе выбора, например` select * from table1, где column1 = 1`.
## Пример использования {#primer-ispolzovaniia}
@ -85,6 +85,12 @@ FROM jdbc_table
└────────┴──────────────┴───────┴────────────────┘
```
``` sql
INSERT INTO jdbc_table(`int_id`, `float`)
SELECT toInt32(number), toFloat32(number * 1.0)
FROM system.numbers
```
## Смотрите также {#smotrite-takzhe}
- [Табличная функция JDBC](../../../engines/table-engines/integrations/jdbc.md).

View File

@ -1,4 +1,4 @@
# LDAP {#external-authenticators-ldap}
# LDAP {#external-authenticators-ldap}
Для аутентификации пользователей ClickHouse можно использовать сервер LDAP. Существуют два подхода:
@ -17,6 +17,7 @@
<yandex>
<!- ... -->
<ldap_servers>
<!- Typical LDAP server. -->
<my_ldap_server>
<host>localhost</host>
<port>636</port>
@ -31,6 +32,18 @@
<tls_ca_cert_dir>/path/to/tls_ca_cert_dir</tls_ca_cert_dir>
<tls_cipher_suite>ECDHE-ECDSA-AES256-GCM-SHA384:ECDHE-RSA-AES256-GCM-SHA384:AES256-GCM-SHA384</tls_cipher_suite>
</my_ldap_server>
<!- Typical Active Directory with configured user DN detection for further role mapping. -->
<my_ad_server>
<host>localhost</host>
<port>389</port>
<bind_dn>EXAMPLE\{user_name}</bind_dn>
<user_dn_detection>
<base_dn>CN=Users,DC=example,DC=com</base_dn>
<search_filter>(&amp;(objectClass=user)(sAMAccountName={user_name}))</search_filter>
</user_dn_detection>
<enable_tls>no</enable_tls>
</my_ad_server>
</ldap_servers>
</yandex>
```
@ -41,9 +54,18 @@
- `host` — имя хоста сервера LDAP или его IP. Этот параметр обязательный и не может быть пустым.
- `port` — порт сервера LDAP. Если настройка `enable_tls` равна `true`, то по умолчанию используется порт `636`, иначе — порт `389`.
- `bind_dn` — шаблон для создания DN для привязки.
- `bind_dn` — шаблон для создания DN подключения.
- При формировании DN все подстроки `{user_name}` в шаблоне будут заменяться на фактическое имя пользователя при каждой попытке аутентификации.
- `verification_cooldown` — промежуток времени (в секундах) после успешной попытки привязки, в течение которого пользователь будет считаться аутентифицированным и сможет выполнять запросы без повторного обращения к серверам LDAP.
- `user_dn_detection` — секция с параметрами LDAP поиска для определения фактического значения DN подключенного пользователя.
- Это в основном используется в фильтрах поиска для дальнейшего сопоставления ролей, когда сервер является Active Directory. Полученный DN пользователя будет использоваться при замене подстрок `{user_dn}` везде, где они разрешены. По умолчанию DN пользователя устанавливается равным DN подключения, но после выполнения поиска он будет обновлен до фактического найденного значения DN пользователя.
- `base_dn` — шаблон для создания базового DN для LDAP поиска.
- При формировании DN все подстроки `{user_name}` и `{bind_dn}` в шаблоне будут заменяться на фактическое имя пользователя и DN подключения соответственно при каждом LDAP поиске.
- `scope` — область LDAP поиска.
- Возможные значения: `base`, `one_level`, `children`, `subtree` (по умолчанию).
- `search_filter` — шаблон для создания фильтра для каждого LDAP поиска.
- При формировании фильтра все подстроки `{user_name}`, `{bind_dn}`, `{user_dn}` и `{base_dn}` в шаблоне будут заменяться на фактическое имя пользователя, DN подключения, DN пользователя и базовый DN соответственно при каждом LDAP поиске.
- Обратите внимание, что специальные символы должны быть правильно экранированы в XML.
- `verification_cooldown` — промежуток времени (в секундах) после успешной попытки подключения, в течение которого пользователь будет считаться аутентифицированным и сможет выполнять запросы без повторного обращения к серверам LDAP.
- Чтобы отключить кеширование и заставить обращаться к серверу LDAP для каждого запроса аутентификации, укажите `0` (значение по умолчанию).
- `enable_tls` — флаг, включающий использование защищенного соединения с сервером LDAP.
- Укажите `no` для использования текстового протокола `ldap://` (не рекомендовано).
@ -106,7 +128,7 @@ CREATE USER my_user IDENTIFIED WITH ldap SERVER 'my_ldap_server';
<yandex>
<!- ... -->
<user_directories>
<!- ... -->
<!- Typical LDAP server. -->
<ldap>
<server>my_ldap_server</server>
<roles>
@ -121,6 +143,18 @@ CREATE USER my_user IDENTIFIED WITH ldap SERVER 'my_ldap_server';
<prefix>clickhouse_</prefix>
</role_mapping>
</ldap>
<!- Typical Active Directory with role mapping that relies on the detected user DN. -->
<ldap>
<server>my_ad_server</server>
<role_mapping>
<base_dn>CN=Users,DC=example,DC=com</base_dn>
<attribute>CN</attribute>
<scope>subtree</scope>
<search_filter>(&amp;(objectClass=group)(member={user_dn}))</search_filter>
<prefix>clickhouse_</prefix>
</role_mapping>
</ldap>
</user_directories>
</yandex>
```
@ -135,14 +169,14 @@ CREATE USER my_user IDENTIFIED WITH ldap SERVER 'my_ldap_server';
- `role_mapping` — секция c параметрами LDAP поиска и правилами отображения.
- При аутентификации пользователя, пока еще связанного с LDAP, производится LDAP поиск с помощью `search_filter` и имени этого пользователя. Для каждой записи, найденной в ходе поиска, выделяется значение указанного атрибута. У каждого атрибута, имеющего указанный префикс, этот префикс удаляется, а остальная часть значения становится именем локальной роли, определенной в ClickHouse, причем предполагается, что эта роль была ранее создана запросом [CREATE ROLE](../../sql-reference/statements/create/role.md#create-role-statement) до этого.
- Внутри одной секции `ldap` может быть несколько секций `role_mapping`. Все они будут применены.
- `base_dn` — шаблон, который используется для создания базового DN для LDAP поиска.
- При формировании DN все подстроки `{user_name}` и `{bind_dn}` в шаблоне будут заменяться на фактическое имя пользователя и DN привязки соответственно при каждом LDAP поиске.
- `scope`Область LDAP поиска.
- `base_dn` — шаблон для создания базового DN для LDAP поиска.
- При формировании DN все подстроки `{user_name}`, `{bind_dn}` и `{user_dn}` в шаблоне будут заменяться на фактическое имя пользователя, DN подключения и DN пользователя соответственно при каждом LDAP поиске.
- `scope`область LDAP поиска.
- Возможные значения: `base`, `one_level`, `children`, `subtree` (по умолчанию).
- `search_filter` — шаблон, который используется для создания фильтра для каждого LDAP поиска.
- при формировании фильтра все подстроки `{user_name}`, `{bind_dn}` и `{base_dn}` в шаблоне будут заменяться на фактическое имя пользователя, DN привязки и базовый DN соответственно при каждом LDAP поиске.
- `search_filter` — шаблон для создания фильтра для каждого LDAP поиска.
- При формировании фильтра все подстроки `{user_name}`, `{bind_dn}`, `{user_dn}` и `{base_dn}` в шаблоне будут заменяться на фактическое имя пользователя, DN подключения, DN пользователя и базовый DN соответственно при каждом LDAP поиске.
- Обратите внимание, что специальные символы должны быть правильно экранированы в XML.
- `attribute` — имя атрибута, значение которого будет возвращаться LDAP поиском.
- `attribute` — имя атрибута, значение которого будет возвращаться LDAP поиском. По умолчанию: `cn`.
- `prefix` — префикс, который, как предполагается, будет находиться перед началом каждой строки в исходном списке строк, возвращаемых LDAP поиском. Префикс будет удален из исходных строк, а сами они будут рассматриваться как имена локальных ролей. По умолчанию: пустая строка.
[Оригинальная статья](https://clickhouse.tech/docs/en/operations/external-authenticators/ldap) <!--hide-->

View File

@ -0,0 +1,107 @@
---
toc_priority: 146
toc_title: intervalLengthSum
---
# intervalLengthSum {#agg_function-intervallengthsum}
Вычисляет длину объединения интервалов (отрезков на числовой оси).
**Синтаксис**
``` sql
intervalLengthSum(start, end)
```
**Аргументы**
- `start` — начальное значение интервала. [Int32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Int64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Float32](../../../sql-reference/data-types/float.md#float32-float64), [Float64](../../../sql-reference/data-types/float.md#float32-float64), [DateTime](../../../sql-reference/data-types/datetime.md#data_type-datetime) или [Date](../../../sql-reference/data-types/date.md#data_type-date).
- `end` — конечное значение интервала. [Int32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Int64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt32](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [UInt64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64), [Float32](../../../sql-reference/data-types/float.md#float32-float64), [Float64](../../../sql-reference/data-types/float.md#float32-float64), [DateTime](../../../sql-reference/data-types/datetime.md#data_type-datetime) или [Date](../../../sql-reference/data-types/date.md#data_type-date).
!!! info "Примечание"
Аргументы должны быть одного типа. В противном случае ClickHouse сгенерирует исключение.
**Возвращаемое значение**
- Длина объединения всех интервалов (отрезков на числовой оси). В зависимости от типа аргумента возвращаемое значение может быть типа [UInt64](../../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-int8-int16-int32-int64) или [Float64](../../../sql-reference/data-types/float.md#float32-float64).
**Примеры**
1. Входная таблица:
``` text
┌─id─┬─start─┬─end─┐
│ a │ 1.1 │ 2.9 │
│ a │ 2.5 │ 3.2 │
│ a │ 4 │ 5 │
└────┴───────┴─────┘
```
В этом примере используются аргументы типа Float32. Функция возвращает значение типа Float64.
Результатом функции будет сумма длин интервалов `[1.1, 3.2]` (объединение `[1.1, 2.9]` и `[2.5, 3.2]`) и `[4, 5]`
Запрос:
``` sql
SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM fl_interval GROUP BY id ORDER BY id;
```
Результат:
``` text
┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐
│ a │ 3.1 │ Float64 │
└────┴───────────────────────────────┴───────────────────────────────────────────┘
```
2. Входная таблица:
``` text
┌─id─┬───────────────start─┬─────────────────end─┐
│ a │ 2020-01-01 01:12:30 │ 2020-01-01 02:10:10 │
│ a │ 2020-01-01 02:05:30 │ 2020-01-01 02:50:31 │
│ a │ 2020-01-01 03:11:22 │ 2020-01-01 03:23:31 │
└────┴─────────────────────┴─────────────────────┘
```
В этом примере используются аргументы типа DateTime. Функция возвращает значение, выраженное в секундах.
Запрос:
``` sql
SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM dt_interval GROUP BY id ORDER BY id;
```
Результат:
``` text
┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐
│ a │ 6610 │ UInt64 │
└────┴───────────────────────────────┴───────────────────────────────────────────┘
```
3. Входная таблица:
``` text
┌─id─┬──────start─┬────────end─┐
│ a │ 2020-01-01 │ 2020-01-04 │
│ a │ 2020-01-12 │ 2020-01-18 │
└────┴────────────┴────────────┘
```
В этом примере используются аргументы типа Date. Функция возвращает значение, выраженное в днях.
Запрос:
``` sql
SELECT id, intervalLengthSum(start, end), toTypeName(intervalLengthSum(start, end)) FROM date_interval GROUP BY id ORDER BY id;
```
Результат:
``` text
┌─id─┬─intervalLengthSum(start, end)─┬─toTypeName(intervalLengthSum(start, end))─┐
│ a │ 9 │ UInt64 │
└────┴───────────────────────────────┴───────────────────────────────────────────┘
```

View File

@ -0,0 +1,39 @@
---
toc_priority: 145
---
# sumKahan {#agg_function-sumKahan}
Вычисляет сумму с использованием [компенсационного суммирования по алгоритму Кэхэна](https://ru.wikipedia.org/wiki/Алгоритм_Кэхэна).
Работает медленнее функции [sum](./sum.md).
Компенсация работает только для [Float](../../../sql-reference/data-types/float.md) типов.
**Синтаксис**
``` sql
sumKahan(x)
```
**Аргументы**
- `x` — Входное значение типа [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md), или [Decimal](../../../sql-reference/data-types/decimal.md).
**Возвращемое значение**
- сумма чисел с типом [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md), ил [Decimal](../../../sql-reference/data-types/decimal.md) зависящим от типа входящих аргументов
**Пример**
Запрос:
``` sql
SELECT sum(0.1), sumKahan(0.1) FROM numbers(10);
```
Результат:
``` text
┌───────────sum(0.1)─┬─sumKahan(0.1)─┐
│ 0.9999999999999999 │ 1 │
└────────────────────┴───────────────┘
```

View File

@ -4,7 +4,7 @@ toc_priority: 109
# topKWeighted {#topkweighted}
Аналогична `topK`, но дополнительно принимает положительный целочисленный параметр `weight`. Каждое значение учитывается `weight` раз при расчёте частоты.
Возвращает массив наиболее часто встречающихся значений в указанном столбце. Результирующий массив упорядочен по убыванию частоты значения (не по самим значениям). Дополнительно учитывается вес значения.
**Синтаксис**
@ -15,11 +15,8 @@ topKWeighted(N)(x, weight)
**Аргументы**
- `N` — количество элементов для выдачи.
**Аргументы**
- `x` — значение.
- `weight` — вес. [UInt8](../../../sql-reference/data-types/int-uint.md).
- `weight` — вес. Каждое значение учитывается `weight` раз при расчёте частоты. [UInt64](../../../sql-reference/data-types/int-uint.md).
**Возвращаемое значение**
@ -41,3 +38,6 @@ SELECT topKWeighted(10)(number, number) FROM numbers(1000)
└───────────────────────────────────────────┘
```
**Смотрите также**
- [topK](../../../sql-reference/aggregate-functions/reference/topk.md)

View File

@ -159,7 +159,7 @@ CREATE DICTIONARY somename (
| Тег | Описание | Обязательный |
|------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------|
| `name` | Имя столбца. | Да |
| `type` | Тип данных ClickHouse.<br/>ClickHouse пытается привести значение из словаря к заданному типу данных. Например, в случае MySQL, в таблице-источнике поле может быть `TEXT`, `VARCHAR`, `BLOB`, но загружено может быть как `String`. <br/>[Nullable](../../../sql-reference/data-types/nullable.md) в настоящее время поддерживается для словарей [Flat](external-dicts-dict-layout.md#flat), [Hashed](external-dicts-dict-layout.md#dicts-external_dicts_dict_layout-hashed), [ComplexKeyHashed](external-dicts-dict-layout.md#complex-key-hashed), [Direct](external-dicts-dict-layout.md#direct), [ComplexKeyDirect](external-dicts-dict-layout.md#complex-key-direct), [RangeHashed](external-dicts-dict-layout.md#range-hashed), [Polygon](external-dicts-dict-polygon.md), [Cache](external-dicts-dict-layout.md#cache), [ComplexKeyCache](external-dicts-dict-layout.md#complex-key-cache), [SSDCache](external-dicts-dict-layout.md#ssd-cache), [SSDComplexKeyCache](external-dicts-dict-layout.md#complex-key-ssd-cache). Для словарей [IPTrie](external-dicts-dict-layout.md#ip-trie) `Nullable`-типы не поддерживаются. | Да |
| `type` | Тип данных ClickHouse: [UInt8](../../../sql-reference/data-types/int-uint.md), [UInt16](../../../sql-reference/data-types/int-uint.md), [UInt32](../../../sql-reference/data-types/int-uint.md), [UInt64](../../../sql-reference/data-types/int-uint.md), [Int8](../../../sql-reference/data-types/int-uint.md), [Int16](../../../sql-reference/data-types/int-uint.md), [Int32](../../../sql-reference/data-types/int-uint.md), [Int64](../../../sql-reference/data-types/int-uint.md), [Float32](../../../sql-reference/data-types/float.md), [Float64](../../../sql-reference/data-types/float.md), [UUID](../../../sql-reference/data-types/uuid.md), [Decimal32](../../../sql-reference/data-types/decimal.md), [Decimal64](../../../sql-reference/data-types/decimal.md), [Decimal128](../../../sql-reference/data-types/decimal.md), [Decimal256](../../../sql-reference/data-types/decimal.md), [String](../../../sql-reference/data-types/string.md).<br/>ClickHouse пытается привести значение из словаря к заданному типу данных. Например, в случае MySQL, в таблице-источнике поле может быть `TEXT`, `VARCHAR`, `BLOB`, но загружено может быть как `String`. <br/>[Nullable](../../../sql-reference/data-types/nullable.md) в настоящее время поддерживается для словарей [Flat](external-dicts-dict-layout.md#flat), [Hashed](external-dicts-dict-layout.md#dicts-external_dicts_dict_layout-hashed), [ComplexKeyHashed](external-dicts-dict-layout.md#complex-key-hashed), [Direct](external-dicts-dict-layout.md#direct), [ComplexKeyDirect](external-dicts-dict-layout.md#complex-key-direct), [RangeHashed](external-dicts-dict-layout.md#range-hashed), [Polygon](external-dicts-dict-polygon.md), [Cache](external-dicts-dict-layout.md#cache), [ComplexKeyCache](external-dicts-dict-layout.md#complex-key-cache), [SSDCache](external-dicts-dict-layout.md#ssd-cache), [SSDComplexKeyCache](external-dicts-dict-layout.md#complex-key-ssd-cache). Для словарей [IPTrie](external-dicts-dict-layout.md#ip-trie) `Nullable`-типы не поддерживаются. | Да |
| `null_value` | Значение по умолчанию для несуществующего элемента.<br/>В примере это пустая строка. Значение [NULL](../../syntax.md#null-literal) можно указывать только для типов `Nullable` (см. предыдущую строку с описанием типов). | Да |
| `expression` | [Выражение](../../syntax.md#syntax-expressions), которое ClickHouse выполняет со значением.<br/>Выражением может быть имя столбца в удаленной SQL базе. Таким образом, вы можете использовать его для создания псевдонима удаленного столбца.<br/><br/>Значение по умолчанию: нет выражения. | Нет |
| <a name="hierarchical-dict-attr"></a> `hierarchical` | Если `true`, то атрибут содержит ключ предка для текущего элемента. Смотрите [Иерархические словари](external-dicts-dict-hierarchical.md).<br/><br/>Значение по умолчанию: `false`. | Нет |

View File

@ -264,6 +264,9 @@ SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp;
└────────────────┘
```
!!! attention "Attention"
`Date` или `DateTime` это возвращаемый тип функций `toStartOf*`, который описан ниже. Несмотря на то, что эти функции могут принимать `DateTime64` в качестве аргумента, если переданное значение типа `DateTime64` выходит за пределы нормального диапазона (с 1925 по 2283 год), то это даст неверный результат.
## toStartOfYear {#tostartofyear}
Округляет дату или дату-с-временем вниз до первого дня года.

View File

@ -319,13 +319,12 @@ GRANT INSERT(x,y) ON db.table TO john
Разрешает выполнять запросы [DROP](misc.md#drop) и [DETACH](misc.md#detach-statement) в соответствии со следующей иерархией привилегий:
- `DROP`. Уровень:
- `DROP`. Уровень: `GROUP`
- `DROP DATABASE`. Уровень: `DATABASE`
- `DROP TABLE`. Уровень: `TABLE`
- `DROP VIEW`. Уровень: `VIEW`
- `DROP DICTIONARY`. Уровень: `DICTIONARY`
### TRUNCATE {#grant-truncate}
Разрешает выполнять запросы [TRUNCATE](../../sql-reference/statements/truncate.md).

View File

@ -6,7 +6,7 @@ toc_title: RENAME
# RENAME Statement {#misc_operations-rename}
## RENAME DATABASE {#misc_operations-rename_database}
Переименование базы данных
Переименовывает базу данных, поддерживается только для движка базы данных Atomic.
```
RENAME DATABASE atomic_database1 TO atomic_database2 [ON CLUSTER cluster]

View File

@ -76,7 +76,7 @@ SYSTEM RELOAD MODELS
**Синтаксис**
```sql
SYSTEM RELOAD MODEL [db.]<model_name>
SYSTEM RELOAD MODEL <model_name>
```
## DROP DNS CACHE {#query_language-system-drop-dns-cache}

View File

@ -5,9 +5,9 @@ toc_title: jdbc
# jdbc {#jdbc}
`jdbc(jdbc_connection_uri, schema, table)` - возвращает таблицу, соединение с которой происходит через JDBC-драйвер.
`jdbc(datasource, schema, table)` - возвращает таблицу, соединение с которой происходит через JDBC-драйвер.
Для работы этой табличной функции требуется отдельно запускать приложение clickhouse-jdbc-bridge.
Для работы этой табличной функции требуется отдельно запускать приложение [clickhouse-jdbc-bridge](https://github.com/ClickHouse/clickhouse-jdbc-bridge).
Данная функция поддерживает Nullable типы (на основании DDL таблицы к которой происходит запрос).
**Пример**
@ -17,10 +17,22 @@ SELECT * FROM jdbc('jdbc:mysql://localhost:3306/?user=root&password=root', 'sche
```
``` sql
SELECT * FROM jdbc('mysql://localhost:3306/?user=root&password=root', 'schema', 'table')
SELECT * FROM jdbc('mysql://localhost:3306/?user=root&password=root', 'select * from schema.table')
```
``` sql
SELECT * FROM jdbc('datasource://mysql-local', 'schema', 'table')
SELECT * FROM jdbc('mysql-dev?p1=233', 'num Int32', 'select toInt32OrZero(''{{p1}}'') as num')
```
``` sql
SELECT *
FROM jdbc('mysql-dev?p1=233', 'num Int32', 'select toInt32OrZero(''{{p1}}'') as num')
```
``` sql
SELECT a.datasource AS server1, b.datasource AS server2, b.name AS db
FROM jdbc('mysql-dev?datasource_column', 'show databases') a
INNER JOIN jdbc('self?datasource_column', 'show databases') b ON a.Database = b.name
```
[Оригинальная статья](https://clickhouse.tech/docs/en/query_language/table_functions/jdbc/) <!--hide-->

View File

@ -122,14 +122,14 @@ FROM s3('https://storage.yandexcloud.net/my-test-bucket-768/big_prefix/file-{000
Запишем данные в файл `test-data.csv.gz`:
``` sql
INSERT INTO s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip')
INSERT INTO FUNCTION s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip')
VALUES ('test-data', 1), ('test-data-2', 2);
```
Запишем данные из существующей таблицы в файл `test-data.csv.gz`:
``` sql
INSERT INTO s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip')
INSERT INTO FUNCTION s3('https://storage.yandexcloud.net/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip')
SELECT name, value FROM existing_table;
```

View File

@ -1,27 +1,24 @@
---
machine_translated: true
machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd
toc_priority: 36
toc_title: HDFS
---
# HDFS {#table_engines-hdfs}
该引擎提供了集成 [Apache Hadoop](https://en.wikipedia.org/wiki/Apache_Hadoop) 生态系统通过允许管理数据 [HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html)通过ClickHouse. 这个引擎是相似的
[文件](../special/file.md#table_engines-file) 和 [URL](../special/url.md#table_engines-url) 引擎但提供Hadoop特定功能。
这个引擎提供了与 [Apache Hadoop](https://en.wikipedia.org/wiki/Apache_Hadoop) 生态系统的集成,允许通过 ClickHouse 管理 [HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html) 上的数据。这个引擎类似于
[文件](../../../engines/table-engines/special/file.md#table_engines-file) 和 [URL](../../../engines/table-engines/special/url.md#table_engines-url) 引擎,但提供Hadoop特定功能。
## 用 {#usage}
## 用 {#usage}
``` sql
ENGINE = HDFS(URI, format)
```
`URI` 参数是HDFS中的整个文件URI。
`format` 参数指定一种可用的文件格式。 执行
`SELECT` 查询时,格式必须支持输入,并执行
`INSERT` queries for output. The available formats are listed in the
[格式](../../../interfaces/formats.md#formats) 科。
路径部分 `URI` 可能包含水珠。 在这种情况下,表将是只读的。
`URI` 参数是 HDFS 中整个文件的 URI。
`format` 参数指定一种可用的文件格式。 执行
`SELECT` 查询时,格式必须支持输入,以及执行
`INSERT` 查询时,格式必须支持输出. 你可以在 [格式](../../../interfaces/formats.md#formats) 章节查看可用的格式。
路径部分 `URI` 可能包含 glob 通配符。 在这种情况下,表将是只读的。
**示例:**
@ -58,20 +55,20 @@ SELECT * FROM hdfs_engine_table LIMIT 2
- 索引。
- 复制。
**路径中的水珠**
**路径中的通配符**
多个路径组件可以具有globs。 对于正在处理的文件应该存在并匹配到整个路径模式。 文件列表确定在 `SELECT` (不在 `CREATE` 时刻)。
多个路径组件可以具有 globs。 对于正在处理的文件应该存在并匹配到整个路径模式。 文件列表的确定是在 `SELECT` 的时候进行(而不是在 `CREATE` 的时候)。
- `*`Substitutes any number of any characters except `/` 包括空字符串。
- `?`Substitutes any single character.
- `{some_string,another_string,yet_another_one}`Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`.
- `{N..M}`Substitutes any number in range from N to M including both borders.
- `*`替代任何数量的任何字符,除了 `/` 以及空字符串。
- `?`代替任何单个字符.
- `{some_string,another_string,yet_another_one}`替代任何字符串 `'some_string', 'another_string', 'yet_another_one'`.
- `{N..M}`替换 N 到 M 范围内的任何数字,包括两个边界的值.
建筑与 `{}` 类似于 [远程](../../../sql-reference/table-functions/remote.md) 表功能
`{}` 的结构类似于 [远程](../../../sql-reference/table-functions/remote.md) 表函数
**示例**
1. 假设我们在HDFS上有几个TSV格式的文件其中包含以下Uri:
1. 假设我们在 HDFS 上有几个 TSV 格式的文件,文件的 URI 如下:
- hdfs://hdfs1:9000/some_dir/some_file_1
- hdfs://hdfs1:9000/some_dir/some_file_2
@ -111,10 +108,98 @@ CREATE TABLE table_with_asterisk (name String, value UInt32) ENGINE = HDFS('hdfs
CREARE TABLE big_table (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV')
```
## 配置 {#configuration}
与 GraphiteMergeTree 类似HDFS 引擎支持使用 ClickHouse 配置文件进行扩展配置。有两个配置键可以使用:全局 (`hdfs`) 和用户级别 (`hdfs_*`)。首先全局配置生效,然后用户级别配置生效 (如果用户级别配置存在) 。
``` xml
<!-- HDFS 引擎类型的全局配置选项 -->
<hdfs>
<hadoop_kerberos_keytab>/tmp/keytab/clickhouse.keytab</hadoop_kerberos_keytab>
<hadoop_kerberos_principal>clickuser@TEST.CLICKHOUSE.TECH</hadoop_kerberos_principal>
<hadoop_security_authentication>kerberos</hadoop_security_authentication>
</hdfs>
<!-- 用户 "root" 的指定配置 -->
<hdfs_root>
<hadoop_kerberos_principal>root@TEST.CLICKHOUSE.TECH</hadoop_kerberos_principal>
</hdfs_root>
```
### 可选配置选项及其默认值的列表
#### libhdfs3 支持的
| **参数** | **默认值** |
| rpc\_client\_connect\_tcpnodelay | true |
| dfs\_client\_read\_shortcircuit | true |
| output\_replace-datanode-on-failure | true |
| input\_notretry-another-node | false |
| input\_localread\_mappedfile | true |
| dfs\_client\_use\_legacy\_blockreader\_local | false |
| rpc\_client\_ping\_interval | 10 * 1000 |
| rpc\_client\_connect\_timeout | 600 * 1000 |
| rpc\_client\_read\_timeout | 3600 * 1000 |
| rpc\_client\_write\_timeout | 3600 * 1000 |
| rpc\_client\_socekt\_linger\_timeout | -1 |
| rpc\_client\_connect\_retry | 10 |
| rpc\_client\_timeout | 3600 * 1000 |
| dfs\_default\_replica | 3 |
| input\_connect\_timeout | 600 * 1000 |
| input\_read\_timeout | 3600 * 1000 |
| input\_write\_timeout | 3600 * 1000 |
| input\_localread\_default\_buffersize | 1 * 1024 * 1024 |
| dfs\_prefetchsize | 10 |
| input\_read\_getblockinfo\_retry | 3 |
| input\_localread\_blockinfo\_cachesize | 1000 |
| input\_read\_max\_retry | 60 |
| output\_default\_chunksize | 512 |
| output\_default\_packetsize | 64 * 1024 |
| output\_default\_write\_retry | 10 |
| output\_connect\_timeout | 600 * 1000 |
| output\_read\_timeout | 3600 * 1000 |
| output\_write\_timeout | 3600 * 1000 |
| output\_close\_timeout | 3600 * 1000 |
| output\_packetpool\_size | 1024 |
| output\_heeartbeat\_interval | 10 * 1000 |
| dfs\_client\_failover\_max\_attempts | 15 |
| dfs\_client\_read\_shortcircuit\_streams\_cache\_size | 256 |
| dfs\_client\_socketcache\_expiryMsec | 3000 |
| dfs\_client\_socketcache\_capacity | 16 |
| dfs\_default\_blocksize | 64 * 1024 * 1024 |
| dfs\_default\_uri | "hdfs://localhost:9000" |
| hadoop\_security\_authentication | "simple" |
| hadoop\_security\_kerberos\_ticket\_cache\_path | "" |
| dfs\_client\_log\_severity | "INFO" |
| dfs\_domain\_socket\_path | "" |
[HDFS 配置参考](https://hawq.apache.org/docs/userguide/2.3.0.0-incubating/reference/HDFSConfigurationParameterReference.html) 也许会解释一些参数的含义.
#### ClickHouse 额外的配置 {#clickhouse-extras}
| **参数** | **默认值** |
|hadoop\_kerberos\_keytab | "" |
|hadoop\_kerberos\_principal | "" |
|hadoop\_kerberos\_kinit\_command | kinit |
#### 限制 {#limitations}
* hadoop\_security\_kerberos\_ticket\_cache\_path 只能在全局配置, 不能指定用户
## Kerberos 支持 {#kerberos-support}
如果 hadoop\_security\_authentication 参数的值为 'kerberos' ClickHouse 将通过 Kerberos 设施进行认证。
[这里的](#clickhouse-extras) 参数和 hadoop\_security\_kerberos\_ticket\_cache\_path 也许会有帮助.
注意,由于 libhdfs3 的限制,只支持老式的方法。
数据节点的安全通信无法由 SASL 保证 ( HADOOP\_SECURE\_DN\_USER 是这种安全方法的一个可靠指标)
使用 tests/integration/test\_storage\_kerberized\_hdfs/hdfs_configs/bootstrap.sh 脚本作为参考。
如果指定了 hadoop\_kerberos\_keytab, hadoop\_kerberos\_principal 或者 hadoop\_kerberos\_kinit\_command ,将会调用 kinit 工具.在此情况下, hadoop\_kerberos\_keytab 和 hadoop\_kerberos\_principal 参数是必须配置的. kinit 工具和 krb5 配置文件是必要的.
## 虚拟列 {#virtual-columns}
- `_path` — Path to the file.
- `_file` — Name of the file.
- `_path`文件路径.
- `_file`文件名.
**另请参阅**

View File

@ -8,7 +8,7 @@ toc_title: JDBC表引擎
允许CH通过 [JDBC](https://en.wikipedia.org/wiki/Java_Database_Connectivity) 连接到外部数据库。
要实现JDBC连接CH需要使用以后台进程运行的程序 [clickhouse-jdbc-bridge](https://github.com/alex-krash/clickhouse-jdbc-bridge)。
要实现JDBC连接CH需要使用以后台进程运行的程序 [clickhouse-jdbc-bridge](https://github.com/ClickHouse/clickhouse-jdbc-bridge)。
该引擎支持 [Nullable](../../../sql-reference/data-types/nullable.md) 数据类型。
@ -20,19 +20,19 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name
(
columns list...
)
ENGINE = JDBC(dbms_uri, external_database, external_table)
ENGINE = JDBC(datasource_uri, external_database, external_table)
```
**引擎参数**
- `dbms_uri` — 外部DBMS的uri.
- `datasource_uri` — 外部DBMS的URI或名字.
格式: `jdbc:<driver_name>://<host_name>:<port>/?user=<username>&password=<password>`.
URI格式: `jdbc:<driver_name>://<host_name>:<port>/?user=<username>&password=<password>`.
MySQL示例: `jdbc:mysql://localhost:3306/?user=root&password=root`.
- `external_database` — 外部DBMS的数据库名.
- `external_table``external_database`中的外部表名.
- `external_table``external_database`中的外部表名或类似`select * from table1 where column1=1`的查询语句.
## 用法示例 {#usage-example}
@ -85,6 +85,12 @@ FROM jdbc_table
└────────┴──────────────┴───────┴────────────────┘
```
``` sql
INSERT INTO jdbc_table(`int_id`, `float`)
SELECT toInt32(number), toFloat32(number * 1.0)
FROM system.numbers
```
## 参见 {#see-also}
- [JDBC表函数](../../../sql-reference/table-functions/jdbc.md).

View File

@ -5,9 +5,9 @@ toc_title: jdbc
# jdbc {#table-function-jdbc}
`jdbc(jdbc_connection_uri, schema, table)` -返回通过JDBC驱动程序连接的表。
`jdbc(datasource, schema, table)` -返回通过JDBC驱动程序连接的表。
此表函数需要单独的 `clickhouse-jdbc-bridge` 程序才能运行。
此表函数需要单独的 [clickhouse-jdbc-bridge](https://github.com/ClickHouse/clickhouse-jdbc-bridge) 程序才能运行。
它支持可空类型基于查询的远程表的DDL
**示例**
@ -17,11 +17,22 @@ SELECT * FROM jdbc('jdbc:mysql://localhost:3306/?user=root&password=root', 'sche
```
``` sql
SELECT * FROM jdbc('mysql://localhost:3306/?user=root&password=root', 'schema', 'table')
SELECT * FROM jdbc('mysql://localhost:3306/?user=root&password=root', 'select * from schema.table')
```
``` sql
SELECT * FROM jdbc('datasource://mysql-local', 'schema', 'table')
SELECT * FROM jdbc('mysql-dev?p1=233', 'num Int32', 'select toInt32OrZero(''{{p1}}'') as num')
```
``` sql
SELECT *
FROM jdbc('mysql-dev?p1=233', 'num Int32', 'select toInt32OrZero(''{{p1}}'') as num')
```
``` sql
SELECT a.datasource AS server1, b.datasource AS server2, b.name AS db
FROM jdbc('mysql-dev?datasource_column', 'show databases') a
INNER JOIN jdbc('self?datasource_column', 'show databases') b ON a.Database = b.name
```
[原始文章](https://clickhouse.tech/docs/en/query_language/table_functions/jdbc/) <!--hide-->

View File

@ -967,13 +967,10 @@ private:
TestHint test_hint(test_mode, all_queries_text);
if (test_hint.clientError() || test_hint.serverError())
processTextAsSingleQuery("SET send_logs_level = 'fatal'");
// Echo all queries if asked; makes for a more readable reference
// file.
if (test_hint.echoQueries())
echo_queries = true;
}
bool echo_query = echo_queries;
/// Several queries separated by ';'.
/// INSERT data is ended by the end of line, not ';'.
/// An exception is VALUES format where we also support semicolon in
@ -1106,9 +1103,20 @@ private:
continue;
}
// Now we know for sure where the query ends.
// Look for the hint in the text of query + insert data + trailing
// comments,
// e.g. insert into t format CSV 'a' -- { serverError 123 }.
// Use the updated query boundaries we just calculated.
TestHint test_hint(test_mode, std::string(this_query_begin, this_query_end - this_query_begin));
// Echo all queries if asked; makes for a more readable reference
// file.
echo_query = test_hint.echoQueries().value_or(echo_query);
try
{
processParsedSingleQuery();
processParsedSingleQuery(echo_query);
}
catch (...)
{
@ -1130,13 +1138,6 @@ private:
adjustQueryEnd(this_query_end, all_queries_end, context->getSettingsRef().max_parser_depth);
}
// Now we know for sure where the query ends.
// Look for the hint in the text of query + insert data + trailing
// comments,
// e.g. insert into t format CSV 'a' -- { serverError 123 }.
// Use the updated query boundaries we just calculated.
TestHint test_hint(test_mode, std::string(this_query_begin, this_query_end - this_query_begin));
// Check whether the error (or its absence) matches the test hints
// (or their absence).
bool error_matches_hint = true;
@ -1336,7 +1337,7 @@ private:
fmt::print(
stderr,
"IAST::clone() is broken for some AST node. This is a bug. The original AST ('dump before fuzz') and its cloned copy ('dump of cloned AST') refer to the same nodes, which must never happen. This means that their parent node doesn't implement clone() correctly.");
"Found error: IAST::clone() is broken for some AST node. This is a bug. The original AST ('dump before fuzz') and its cloned copy ('dump of cloned AST') refer to the same nodes, which must never happen. This means that their parent node doesn't implement clone() correctly.");
exit(1);
}
@ -1461,7 +1462,7 @@ private:
const auto text_3 = ast_3->formatForErrorMessage();
if (text_3 != text_2)
{
fmt::print(stderr, "The query formatting is broken.\n");
fmt::print(stderr, "Found error: The query formatting is broken.\n");
printChangedSettings();
@ -1547,14 +1548,14 @@ private:
// 'query_to_send' -- the query text that is sent to server,
// 'full_query' -- for INSERT queries, contains the query and the data that
// follow it. Its memory is referenced by ASTInsertQuery::begin, end.
void processParsedSingleQuery()
void processParsedSingleQuery(std::optional<bool> echo_query = {})
{
resetOutput();
client_exception.reset();
server_exception.reset();
have_error = false;
if (echo_queries)
if (echo_query.value_or(echo_queries))
{
writeString(full_query, std_out);
writeChar('\n', std_out);

View File

@ -325,14 +325,14 @@ void QueryFuzzer::fuzzColumnLikeExpressionList(IAST * ast)
// the generic recursion into IAST.children.
}
void QueryFuzzer::fuzzWindowFrame(WindowFrame & frame)
void QueryFuzzer::fuzzWindowFrame(ASTWindowDefinition & def)
{
switch (fuzz_rand() % 40)
{
case 0:
{
const auto r = fuzz_rand() % 3;
frame.type = r == 0 ? WindowFrame::FrameType::Rows
def.frame_type = r == 0 ? WindowFrame::FrameType::Rows
: r == 1 ? WindowFrame::FrameType::Range
: WindowFrame::FrameType::Groups;
break;
@ -340,44 +340,65 @@ void QueryFuzzer::fuzzWindowFrame(WindowFrame & frame)
case 1:
{
const auto r = fuzz_rand() % 3;
frame.begin_type = r == 0 ? WindowFrame::BoundaryType::Unbounded
def.frame_begin_type = r == 0 ? WindowFrame::BoundaryType::Unbounded
: r == 1 ? WindowFrame::BoundaryType::Current
: WindowFrame::BoundaryType::Offset;
if (def.frame_begin_type == WindowFrame::BoundaryType::Offset)
{
// The offsets are fuzzed normally through 'children'.
def.frame_begin_offset
= std::make_shared<ASTLiteral>(getRandomField(0));
}
else
{
def.frame_begin_offset = nullptr;
}
break;
}
case 2:
{
const auto r = fuzz_rand() % 3;
frame.end_type = r == 0 ? WindowFrame::BoundaryType::Unbounded
def.frame_end_type = r == 0 ? WindowFrame::BoundaryType::Unbounded
: r == 1 ? WindowFrame::BoundaryType::Current
: WindowFrame::BoundaryType::Offset;
break;
}
case 3:
{
frame.begin_offset = getRandomField(0).get<Int64>();
break;
}
case 4:
{
frame.end_offset = getRandomField(0).get<Int64>();
if (def.frame_end_type == WindowFrame::BoundaryType::Offset)
{
def.frame_end_offset
= std::make_shared<ASTLiteral>(getRandomField(0));
}
else
{
def.frame_end_offset = nullptr;
}
break;
}
case 5:
{
frame.begin_preceding = fuzz_rand() % 2;
def.frame_begin_preceding = fuzz_rand() % 2;
break;
}
case 6:
{
frame.end_preceding = fuzz_rand() % 2;
def.frame_end_preceding = fuzz_rand() % 2;
break;
}
default:
break;
}
frame.is_default = (frame == WindowFrame{});
if (def.frame_type == WindowFrame::FrameType::Range
&& def.frame_begin_type == WindowFrame::BoundaryType::Unbounded
&& def.frame_begin_preceding
&& def.frame_end_type == WindowFrame::BoundaryType::Current)
{
def.frame_is_default = true; /* NOLINT clang-tidy could you just shut up please */
}
else
{
def.frame_is_default = false;
}
}
void QueryFuzzer::fuzz(ASTs & asts)
@ -464,7 +485,7 @@ void QueryFuzzer::fuzz(ASTPtr & ast)
auto & def = fn->window_definition->as<ASTWindowDefinition &>();
fuzzColumnLikeExpressionList(def.partition_by.get());
fuzzOrderByList(def.order_by.get());
fuzzWindowFrame(def.frame);
fuzzWindowFrame(def);
}
fuzz(fn->children);

View File

@ -17,7 +17,7 @@ namespace DB
class ASTExpressionList;
class ASTOrderByElement;
struct WindowFrame;
struct ASTWindowDefinition;
/*
* This is an AST-based query fuzzer that makes random modifications to query
@ -69,7 +69,7 @@ struct QueryFuzzer
void fuzzOrderByElement(ASTOrderByElement * elem);
void fuzzOrderByList(IAST * ast);
void fuzzColumnLikeExpressionList(IAST * ast);
void fuzzWindowFrame(WindowFrame & frame);
void fuzzWindowFrame(ASTWindowDefinition & def);
void fuzz(ASTs & asts);
void fuzz(ASTPtr & ast);
void collectFuzzInfoMain(const ASTPtr ast);

View File

@ -11,9 +11,35 @@
namespace DB
{
/// Checks expected server and client error codes in testmode.
/// To enable it add special comment after the query: "-- { serverError 60 }" or "-- { clientError 20 }".
/// Also you can enable echoing all queries by writing "-- { echo }".
/// Checks expected server and client error codes in --testmode.
///
/// The following comment hints are supported:
///
/// - "-- { serverError 60 }" -- in case of you are expecting server error.
///
/// - "-- { clientError 20 }" -- in case of you are expecting client error.
///
/// Remember that the client parse the query first (not the server), so for
/// example if you are expecting syntax error, then you should use
/// clientError not serverError.
///
/// Examples:
///
/// - echo 'select / -- { clientError 62 }' | clickhouse-client --testmode -nm
///
// Here the client parses the query but it is incorrect, so it expects
/// SYNTAX_ERROR (62).
///
/// - echo 'select foo -- { serverError 47 }' | clickhouse-client --testmode -nm
///
/// But here the query is correct, but there is no such column "foo", so it
/// is UNKNOWN_IDENTIFIER server error.
///
/// The following hints will control the query echo mode (i.e print each query):
///
/// - "-- { echo }"
/// - "-- { echoOn }"
/// - "-- { echoOff }"
class TestHint
{
public:
@ -59,13 +85,13 @@ public:
int serverError() const { return server_error; }
int clientError() const { return client_error; }
bool echoQueries() const { return echo; }
std::optional<bool> echoQueries() const { return echo; }
private:
const String & query;
int server_error = 0;
int client_error = 0;
bool echo = false;
std::optional<bool> echo;
void parse(const String & hint, bool is_leading_hint)
{
@ -88,7 +114,11 @@ private:
}
if (item == "echo")
echo = true;
echo.emplace(true);
if (item == "echoOn")
echo.emplace(true);
if (item == "echoOff")
echo.emplace(false);
}
}

File diff suppressed because it is too large Load Diff

View File

@ -18,12 +18,13 @@ public:
ClusterCopier(const String & task_path_,
const String & host_id_,
const String & proxy_database_name_,
ContextMutablePtr context_)
ContextMutablePtr context_,
Poco::Logger * log_)
: WithMutableContext(context_),
task_zookeeper_path(task_path_),
host_id(host_id_),
working_database_name(proxy_database_name_),
log(&Poco::Logger::get("ClusterCopier")) {}
log(log_) {}
void init();
@ -117,14 +118,14 @@ protected:
TaskStatus tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name);
/// Removes MATERIALIZED and ALIAS columns from create table query
static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast);
static ASTPtr removeAliasMaterializedAndTTLColumnsFromCreateQuery(const ASTPtr & query_ast, bool allow_to_copy_alias_and_materialized_columns);
bool tryDropPartitionPiece(ShardPartition & task_partition, size_t current_piece_number,
const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock);
static constexpr UInt64 max_table_tries = 3;
static constexpr UInt64 max_shard_partition_tries = 3;
static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 3;
static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 10;
bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table);
@ -189,9 +190,7 @@ protected:
const ClusterPtr & cluster,
const String & query,
const Settings & current_settings,
PoolMode pool_mode = PoolMode::GET_ALL,
ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_SHARD,
UInt64 max_successful_executions_per_shard = 0) const;
ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_SHARD) const;
private:
String task_zookeeper_path;
@ -208,7 +207,6 @@ private:
ConfigurationPtr task_cluster_initial_config;
ConfigurationPtr task_cluster_current_config;
Coordination::Stat task_description_current_stat{};
std::unique_ptr<TaskCluster> task_cluster;

View File

@ -22,8 +22,9 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self)
config_xml_path = config().getString("config-file");
task_path = config().getString("task-path");
log_level = config().getString("log-level", "trace");
log_level = config().getString("log-level", "info");
is_safe_mode = config().has("safe-mode");
is_status_mode = config().has("status");
if (config().has("copy-fault-probability"))
copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0);
if (config().has("move-fault-probability"))
@ -97,6 +98,7 @@ void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options)
.argument("base-dir").binding("base-dir"));
options.addOption(Poco::Util::Option("experimental-use-sample-offset", "", "Use SAMPLE OFFSET query instead of cityHash64(PRIMARY KEY) % n == k")
.argument("experimental-use-sample-offset").binding("experimental-use-sample-offset"));
options.addOption(Poco::Util::Option("status", "", "Get for status for current execution").binding("status"));
using Me = std::decay_t<decltype(*this)>;
options.addOption(Poco::Util::Option("help", "", "produce this help message").binding("help")
@ -106,6 +108,25 @@ void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options)
void ClusterCopierApp::mainImpl()
{
/// Status command
{
if (is_status_mode)
{
SharedContextHolder shared_context = Context::createShared();
auto context = Context::createGlobal(shared_context.get());
context->makeGlobalContext();
SCOPE_EXIT_SAFE(context->shutdown());
auto zookeeper = context->getZooKeeper();
auto status_json = zookeeper->get(task_path + "/status");
LOG_INFO(&logger(), "{}", status_json);
std::cout << status_json << std::endl;
context->resetZooKeeper();
return;
}
}
StatusFile status_file(process_path + "/status", StatusFile::write_full_info);
ThreadStatus thread_status;
@ -136,7 +157,7 @@ void ClusterCopierApp::mainImpl()
/// Initialize query scope just in case.
CurrentThread::QueryScope query_scope(context);
auto copier = std::make_unique<ClusterCopier>(task_path, host_id, default_database, context);
auto copier = std::make_unique<ClusterCopier>(task_path, host_id, default_database, context, log);
copier->setSafeMode(is_safe_mode);
copier->setCopyFaultProbability(copy_fault_probability);
copier->setMoveFaultProbability(move_fault_probability);

View File

@ -76,8 +76,9 @@ private:
std::string config_xml_path;
std::string task_path;
std::string log_level = "trace";
std::string log_level = "info";
bool is_safe_mode = false;
bool is_status_mode = false;
double copy_fault_probability = 0.0;
double move_fault_probability = 0.0;
bool is_help = false;

View File

@ -0,0 +1,65 @@
#pragma once
#include <Poco/JSON/Parser.h>
#include <Poco/JSON/JSON.h>
#include <Poco/JSON/Object.h>
#include <Poco/JSON/Stringifier.h>
#include <unordered_map>
#include <memory>
#include <string>
#include <iostream>
namespace DB
{
class StatusAccumulator
{
public:
struct TableStatus
{
size_t all_partitions_count;
size_t processed_partitions_count;
};
using Map = std::unordered_map<std::string, TableStatus>;
using MapPtr = std::shared_ptr<Map>;
static MapPtr fromJSON(std::string state_json)
{
Poco::JSON::Parser parser;
auto state = parser.parse(state_json).extract<Poco::JSON::Object::Ptr>();
MapPtr result_ptr = std::make_shared<Map>();
for (const auto & table_name : state->getNames())
{
auto table_status_json = state->getValue<std::string>(table_name);
auto table_status = parser.parse(table_status_json).extract<Poco::JSON::Object::Ptr>();
/// Map entry will be created if it is absent
auto & map_table_status = (*result_ptr)[table_name];
map_table_status.all_partitions_count += table_status->getValue<size_t>("all_partitions_count");
map_table_status.processed_partitions_count += table_status->getValue<size_t>("processed_partitions_count");
}
return result_ptr;
}
static std::string serializeToJSON(MapPtr statuses)
{
Poco::JSON::Object result_json;
for (const auto & [table_name, table_status] : *statuses)
{
Poco::JSON::Object status_json;
status_json.set("all_partitions_count", table_status.all_partitions_count);
status_json.set("processed_partitions_count", table_status.processed_partitions_count);
result_json.set(table_name, status_json);
}
std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
oss.exceptions(std::ios::failbit);
Poco::JSON::Stringifier::stringify(result_json, oss);
auto result = oss.str();
return result;
}
};
}

View File

@ -77,6 +77,8 @@ inline void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfigurat
if (config.has(prefix + "settings"))
settings_common.loadSettingsFromConfig(prefix + "settings", config);
settings_common.prefer_localhost_replica = 0;
settings_pull = settings_common;
if (config.has(prefix + "settings_pull"))
settings_pull.loadSettingsFromConfig(prefix + "settings_pull", config);
@ -92,11 +94,15 @@ inline void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfigurat
/// Override important settings
settings_pull.readonly = 1;
settings_push.insert_distributed_sync = 1;
settings_pull.prefer_localhost_replica = false;
settings_push.insert_distributed_sync = true;
settings_push.prefer_localhost_replica = false;
set_default_value(settings_pull.load_balancing, LoadBalancing::NEAREST_HOSTNAME);
set_default_value(settings_pull.max_threads, 1);
set_default_value(settings_pull.max_block_size, 8192UL);
set_default_value(settings_pull.preferred_block_size_bytes, 0);
set_default_value(settings_push.insert_distributed_timeout, 0);
set_default_value(settings_push.replication_alter_partitions_sync, 2);
}

View File

@ -36,27 +36,33 @@ struct TaskTable
String getPartitionAttachIsDonePath(const String & partition_name) const;
String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const;
String getPartitionPiecePath(const String & partition_name, size_t piece_number) const;
String getCertainPartitionIsDirtyPath(const String & partition_name) const;
String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const;
String getCertainPartitionPieceIsDirtyPath(const String & partition_name, size_t piece_number) const;
String getCertainPartitionIsCleanedPath(const String & partition_name) const;
String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const;
String getCertainPartitionPieceIsCleanedPath(const String & partition_name, size_t piece_number) const;
String getCertainPartitionTaskStatusPath(const String & partition_name) const;
String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const;
String getCertainPartitionPieceTaskStatusPath(const String & partition_name, size_t piece_number) const;
bool isReplicatedTable() const { return is_replicated_table; }
/// These nodes are used for check-status option
String getStatusAllPartitionCount() const;
String getStatusProcessedPartitionsCount() const;
/// Partitions will be split into number-of-splits pieces.
/// Each piece will be copied independently. (10 by default)
size_t number_of_splits;
bool allow_to_copy_alias_and_materialized_columns{false};
bool allow_to_drop_target_partitions{false};
String name_in_config;
/// Used as task ID
@ -83,7 +89,7 @@ struct TaskTable
String engine_push_zk_path;
bool is_replicated_table;
ASTPtr rewriteReplicatedCreateQueryToPlain();
ASTPtr rewriteReplicatedCreateQueryToPlain() const;
/*
* A Distributed table definition used to split data
@ -181,6 +187,7 @@ struct TaskShard
/// Last CREATE TABLE query of the table of the shard
ASTPtr current_pull_table_create_query;
ASTPtr current_push_table_create_query;
/// Internal distributed tables
DatabaseAndTableName table_read_shard;
@ -242,6 +249,16 @@ inline String TaskTable::getCertainPartitionPieceTaskStatusPath(const String & p
return getPartitionPiecePath(partition_name, piece_number) + "/shards";
}
inline String TaskTable::getStatusAllPartitionCount() const
{
return task_cluster.task_zookeeper_path + "/status/all_partitions_count";
}
inline String TaskTable::getStatusProcessedPartitionsCount() const
{
return task_cluster.task_zookeeper_path + "/status/processed_partitions_count";
}
inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config,
const String & prefix_, const String & table_key)
: task_cluster(parent)
@ -250,7 +267,10 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf
name_in_config = table_key;
number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10);
number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 3);
allow_to_copy_alias_and_materialized_columns = config.getBool(table_prefix + "allow_to_copy_alias_and_materialized_columns", false);
allow_to_drop_target_partitions = config.getBool(table_prefix + "allow_to_drop_target_partitions", false);
cluster_pull_name = config.getString(table_prefix + "cluster_pull");
cluster_push_name = config.getString(table_prefix + "cluster_push");
@ -343,7 +363,7 @@ inline void TaskTable::initShards(RandomEngine && random_engine)
std::uniform_int_distribution<UInt8> get_urand(0, std::numeric_limits<UInt8>::max());
// Compute the priority
for (auto & shard_info : cluster_pull->getShardsInfo())
for (const auto & shard_info : cluster_pull->getShardsInfo())
{
TaskShardPtr task_shard = std::make_shared<TaskShard>(*this, shard_info);
const auto & replicas = cluster_pull->getShardsAddresses().at(task_shard->indexInCluster());
@ -369,7 +389,7 @@ inline void TaskTable::initShards(RandomEngine && random_engine)
local_shards.assign(all_shards.begin(), it_first_remote);
}
inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain()
inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain() const
{
ASTPtr prev_engine_push_ast = engine_push_ast->clone();
@ -383,9 +403,15 @@ inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain()
{
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());
/// In some cases of Atomic database engine usage ReplicatedMergeTree tables
/// could be created without arguments.
if (!replicated_table_arguments.empty())
{
/// 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();
@ -400,7 +426,7 @@ inline String DB::TaskShard::getDescription() const
inline String DB::TaskShard::getHostNameExample() const
{
auto & replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster());
const auto & replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster());
return replicas.at(0).readableString();
}

View File

@ -19,6 +19,7 @@
#include <Common/StringUtils/StringUtils.h>
#include <Common/getHashOfLoadedBinary.h>
#include <Common/IO.h>
#include <common/phdr_cache.h>
#include <ext/scope_guard.h>
@ -172,11 +173,11 @@ enum class InstructionFail
AVX512 = 8
};
std::pair<const char *, size_t> instructionFailToString(InstructionFail fail)
auto instructionFailToString(InstructionFail fail)
{
switch (fail)
{
#define ret(x) return std::make_pair(x, ARRAY_SIZE(x) - 1)
#define ret(x) return std::make_tuple(STDERR_FILENO, x, ARRAY_SIZE(x) - 1)
case InstructionFail::NONE:
ret("NONE");
case InstructionFail::SSE3:
@ -260,28 +261,12 @@ void checkRequiredInstructionsImpl(volatile InstructionFail & fail)
fail = InstructionFail::NONE;
}
/// This function is safe to use in static initializers.
void writeErrorLen(const char * data, size_t size)
{
while (size != 0)
{
ssize_t res = ::write(STDERR_FILENO, data, size);
if ((-1 == res || 0 == res) && errno != EINTR)
_Exit(1);
if (res > 0)
{
data += res;
size -= res;
}
}
}
/// Macros to avoid using strlen(), since it may fail if SSE is not supported.
#define writeError(data) do \
{ \
static_assert(__builtin_constant_p(data)); \
writeErrorLen(data, ARRAY_SIZE(data) - 1); \
if (!writeRetry(STDERR_FILENO, data, ARRAY_SIZE(data) - 1)) \
_Exit(1); \
} while (false)
/// Check SSE and others instructions availability. Calls exit on fail.
@ -310,7 +295,8 @@ void checkRequiredInstructions()
if (sigsetjmp(jmpbuf, 1))
{
writeError("Instruction check fail. The CPU does not support ");
std::apply(writeErrorLen, instructionFailToString(fail));
if (!std::apply(writeRetry, instructionFailToString(fail)))
_Exit(1);
writeError(" instruction set.\n");
_Exit(1);
}

View File

@ -105,19 +105,19 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ
{
const bool external_table_functions_use_nulls = Poco::NumberParser::parseBool(params.get("external_table_functions_use_nulls", "false"));
auto connection = ODBCConnectionFactory::instance().get(
auto connection_holder = ODBCConnectionFactory::instance().get(
validateODBCConnectionString(connection_string),
getContext()->getSettingsRef().odbc_bridge_connection_pool_size);
nanodbc::catalog catalog(connection->get());
std::string catalog_name;
/// In XDBC tables it is allowed to pass either database_name or schema_name in table definion, but not both of them.
/// They both are passed as 'schema' parameter in request URL, so it is not clear whether it is database_name or schema_name passed.
/// If it is schema_name then we know that database is added in odbc.ini. But if we have database_name as 'schema',
/// it is not guaranteed. For nanodbc database_name must be either in odbc.ini or passed as catalog_name.
auto get_columns = [&]()
auto get_columns = [&](nanodbc::connection & connection)
{
nanodbc::catalog catalog(connection);
std::string catalog_name;
nanodbc::catalog::tables tables = catalog.find_tables(table_name, /* type = */ "", /* schema = */ "", /* catalog = */ schema_name);
if (tables.next())
{
@ -137,7 +137,9 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table {} not found", schema_name.empty() ? table_name : schema_name + '.' + table_name);
};
nanodbc::catalog::columns columns_definition = get_columns();
nanodbc::catalog::columns columns_definition = execute<nanodbc::catalog::columns>(
std::move(connection_holder),
[&](nanodbc::connection & connection) { return get_columns(connection); });
NamesAndTypesList columns;
while (columns_definition.next())

View File

@ -46,7 +46,7 @@ void IdentifierQuoteHandler::handleRequest(HTTPServerRequest & request, HTTPServ
validateODBCConnectionString(connection_string),
getContext()->getSettingsRef().odbc_bridge_connection_pool_size);
auto identifier = getIdentifierQuote(connection->get());
auto identifier = getIdentifierQuote(std::move(connection));
WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout);
try

View File

@ -108,7 +108,7 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
try
{
auto connection = ODBCConnectionFactory::instance().get(
auto connection_handler = ODBCConnectionFactory::instance().get(
validateODBCConnectionString(connection_string),
getContext()->getSettingsRef().odbc_bridge_connection_pool_size);
@ -130,12 +130,12 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
auto quoting_style = IdentifierQuotingStyle::None;
#if USE_ODBC
quoting_style = getQuotingStyle(connection->get());
quoting_style = getQuotingStyle(connection_handler);
#endif
auto & read_buf = request.getStream();
auto input_format = FormatFactory::instance().getInput(format, read_buf, *sample_block, getContext(), max_block_size);
auto input_stream = std::make_shared<InputStreamFromInputFormat>(input_format);
ODBCBlockOutputStream output_stream(std::move(connection), db_name, table_name, *sample_block, getContext(), quoting_style);
ODBCBlockOutputStream output_stream(std::move(connection_handler), db_name, table_name, *sample_block, getContext(), quoting_style);
copyData(*input_stream, output_stream);
writeStringBinary("Ok.", out);
}
@ -145,7 +145,7 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
LOG_TRACE(log, "Query: {}", query);
BlockOutputStreamPtr writer = FormatFactory::instance().getOutputStreamParallelIfPossible(format, out, *sample_block, getContext());
ODBCBlockInputStream inp(std::move(connection), query, *sample_block, max_block_size);
ODBCBlockInputStream inp(std::move(connection_handler), query, *sample_block, max_block_size);
copyData(inp, *writer);
}
}

View File

@ -21,13 +21,14 @@ namespace ErrorCodes
ODBCBlockInputStream::ODBCBlockInputStream(
nanodbc::ConnectionHolderPtr connection, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_)
nanodbc::ConnectionHolderPtr connection_holder, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_)
: log(&Poco::Logger::get("ODBCBlockInputStream"))
, max_block_size{max_block_size_}
, query(query_str)
{
description.init(sample_block);
result = execute(connection->get(), NANODBC_TEXT(query));
result = execute<nanodbc::result>(connection_holder,
[&](nanodbc::connection & connection) { return execute(connection, query); });
}

View File

@ -40,14 +40,14 @@ namespace
}
}
ODBCBlockOutputStream::ODBCBlockOutputStream(nanodbc::ConnectionHolderPtr connection_,
ODBCBlockOutputStream::ODBCBlockOutputStream(nanodbc::ConnectionHolderPtr connection_holder_,
const std::string & remote_database_name_,
const std::string & remote_table_name_,
const Block & sample_block_,
ContextPtr local_context_,
IdentifierQuotingStyle quoting_)
: log(&Poco::Logger::get("ODBCBlockOutputStream"))
, connection(std::move(connection_))
, connection_holder(std::move(connection_holder_))
, db_name(remote_database_name_)
, table_name(remote_table_name_)
, sample_block(sample_block_)
@ -69,7 +69,8 @@ void ODBCBlockOutputStream::write(const Block & block)
writer->write(block);
std::string query = getInsertQuery(db_name, table_name, block.getColumnsWithTypeAndName(), quoting) + values_buf.str();
execute(connection->get(), query);
execute<void>(connection_holder,
[&](nanodbc::connection & connection) { execute(connection, query); });
}
}

View File

@ -29,7 +29,7 @@ public:
private:
Poco::Logger * log;
nanodbc::ConnectionHolderPtr connection;
nanodbc::ConnectionHolderPtr connection_holder;
std::string db_name;
std::string table_name;
Block sample_block;

View File

@ -21,14 +21,27 @@ using ConnectionPtr = std::unique_ptr<nanodbc::connection>;
using Pool = BorrowedObjectPool<ConnectionPtr>;
using PoolPtr = std::shared_ptr<Pool>;
static constexpr inline auto ODBC_CONNECT_TIMEOUT = 100;
class ConnectionHolder
{
public:
ConnectionHolder(PoolPtr pool_, ConnectionPtr connection_) : pool(pool_), connection(std::move(connection_)) {}
ConnectionHolder(PoolPtr pool_,
ConnectionPtr connection_,
const String & connection_string_)
: pool(pool_)
, connection(std::move(connection_))
, connection_string(connection_string_)
{
}
ConnectionHolder(const ConnectionHolder & other) = delete;
~ConnectionHolder() { pool->returnObject(std::move(connection)); }
~ConnectionHolder()
{
pool->returnObject(std::move(connection));
}
nanodbc::connection & get() const
{
@ -36,12 +49,19 @@ public:
return *connection;
}
void updateConnection()
{
connection = std::make_unique<nanodbc::connection>(connection_string, ODBC_CONNECT_TIMEOUT);
}
private:
PoolPtr pool;
ConnectionPtr connection;
const String & connection_string;
};
using ConnectionHolderPtr = std::unique_ptr<ConnectionHolder>;
using ConnectionHolderPtr = std::shared_ptr<ConnectionHolder>;
}
@ -51,6 +71,26 @@ namespace DB
static constexpr inline auto ODBC_CONNECT_TIMEOUT = 100;
static constexpr inline auto ODBC_POOL_WAIT_TIMEOUT = 10000;
template <typename T>
T execute(nanodbc::ConnectionHolderPtr connection_holder, std::function<T(nanodbc::connection &)> query_func)
{
try
{
return query_func(connection_holder->get());
}
catch (const nanodbc::database_error & e)
{
/// SQLState, connection related errors start with 08S0.
if (e.state().starts_with("08S0"))
{
connection_holder->updateConnection();
return query_func(connection_holder->get());
}
throw;
}
}
class ODBCConnectionFactory final : private boost::noncopyable
{
public:
@ -77,15 +117,16 @@ public:
try
{
if (!connection || !connection->connected())
if (!connection)
connection = std::make_unique<nanodbc::connection>(connection_string, ODBC_CONNECT_TIMEOUT);
}
catch (...)
{
pool->returnObject(std::move(connection));
throw;
}
return std::make_unique<nanodbc::ConnectionHolder>(factory[connection_string], std::move(connection));
return std::make_unique<nanodbc::ConnectionHolder>(factory[connection_string], std::move(connection), connection_string);
}
private:

View File

@ -18,9 +18,10 @@ namespace DB
{
namespace
{
bool isSchemaAllowed(nanodbc::connection & connection)
bool isSchemaAllowed(nanodbc::ConnectionHolderPtr connection_holder)
{
uint32_t result = connection.get_info<uint32_t>(SQL_SCHEMA_USAGE);
uint32_t result = execute<uint32_t>(connection_holder,
[&](nanodbc::connection & connection) { return connection.get_info<uint32_t>(SQL_SCHEMA_USAGE); });
return result != 0;
}
}
@ -53,7 +54,7 @@ void SchemaAllowedHandler::handleRequest(HTTPServerRequest & request, HTTPServer
validateODBCConnectionString(connection_string),
getContext()->getSettingsRef().odbc_bridge_connection_pool_size);
bool result = isSchemaAllowed(connection->get());
bool result = isSchemaAllowed(std::move(connection));
WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout);
try

View File

@ -16,12 +16,13 @@ namespace ErrorCodes
}
std::string getIdentifierQuote(nanodbc::connection & connection)
std::string getIdentifierQuote(nanodbc::ConnectionHolderPtr connection_holder)
{
std::string quote;
try
{
quote = connection.get_info<std::string>(SQL_IDENTIFIER_QUOTE_CHAR);
quote = execute<std::string>(connection_holder,
[&](nanodbc::connection & connection) { return connection.get_info<std::string>(SQL_IDENTIFIER_QUOTE_CHAR); });
}
catch (...)
{
@ -33,7 +34,7 @@ std::string getIdentifierQuote(nanodbc::connection & connection)
}
IdentifierQuotingStyle getQuotingStyle(nanodbc::connection & connection)
IdentifierQuotingStyle getQuotingStyle(nanodbc::ConnectionHolderPtr connection)
{
auto identifier_quote = getIdentifierQuote(connection);
if (identifier_quote.length() == 0)

View File

@ -6,15 +6,14 @@
#include <Poco/Logger.h>
#include <Poco/Net/HTTPRequestHandler.h>
#include <Parsers/IdentifierQuotingStyle.h>
#include <nanodbc/nanodbc.h>
#include "ODBCConnectionFactory.h"
namespace DB
{
std::string getIdentifierQuote(nanodbc::connection & connection);
IdentifierQuotingStyle getQuotingStyle(nanodbc::connection & connection);
std::string getIdentifierQuote(nanodbc::ConnectionHolderPtr connection_holder);
IdentifierQuotingStyle getQuotingStyle(nanodbc::ConnectionHolderPtr connection);
}

View File

@ -1,7 +1,7 @@
#include <AggregateFunctions/AggregateFunctionArray.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <Common/typeid_cast.h>
#include "registerAggregateFunctions.h"
namespace DB
{

View File

@ -2,7 +2,6 @@
#include <AggregateFunctions/AggregateFunctionBitwise.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -1,7 +1,6 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionBoundingRatio.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -3,7 +3,6 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -1,11 +1,9 @@
#include <Common/StringUtils/StringUtils.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include "registerAggregateFunctions.h"
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -1,7 +1,6 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionCount.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -100,7 +100,7 @@ public:
}
/// Reset the state to specified value. This function is not the part of common interface.
void set(AggregateDataPtr __restrict place, UInt64 new_count)
void set(AggregateDataPtr __restrict place, UInt64 new_count) const
{
data(place).count = new_count;
}

View File

@ -2,7 +2,7 @@
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <AggregateFunctions/Helpers.h>
#include <Common/typeid_cast.h>
#include "registerAggregateFunctions.h"
namespace DB
{

View File

@ -2,7 +2,6 @@
#include <AggregateFunctions/AggregateFunctionEntropy.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -17,7 +16,8 @@ namespace ErrorCodes
namespace
{
AggregateFunctionPtr createAggregateFunctionEntropy(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
AggregateFunctionPtr createAggregateFunctionEntropy(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
if (argument_types.empty())

View File

@ -17,10 +17,10 @@
#include <Common/CurrentThread.h>
#include <Poco/String.h>
#include "registerAggregateFunctions.h"
#include <Functions/FunctionFactory.h>
namespace DB
{
struct Settings;
@ -95,7 +95,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
// nullability themselves. Another special case is functions from Nothing
// that are rewritten to AggregateFunctionNothing, in this case
// nested_function is nullptr.
if (nested_function && nested_function->asWindowFunction())
if (nested_function && nested_function->isOnlyWindowFunction())
{
return nested_function;
}

View File

@ -1,12 +1,10 @@
#include <AggregateFunctions/AggregateFunctionForEach.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <Common/typeid_cast.h>
#include "registerAggregateFunctions.h"
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -4,7 +4,6 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -50,7 +49,8 @@ inline AggregateFunctionPtr createAggregateFunctionGroupArrayImpl(const DataType
}
AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
AggregateFunctionPtr createAggregateFunctionGroupArray(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertUnary(name, argument_types);

View File

@ -2,7 +2,6 @@
#include <AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -17,7 +16,8 @@ namespace ErrorCodes
namespace
{
AggregateFunctionPtr createAggregateFunctionGroupArrayInsertAt(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
AggregateFunctionPtr createAggregateFunctionGroupArrayInsertAt(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertBinary(name, argument_types);

View File

@ -5,7 +5,6 @@
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -58,7 +57,8 @@ inline AggregateFunctionPtr createAggregateFunctionMovingImpl(const std::string
}
template <template <typename, typename> class Function>
AggregateFunctionPtr createAggregateFunctionMoving(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
AggregateFunctionPtr createAggregateFunctionMoving(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertUnary(name, argument_types);

View File

@ -2,11 +2,11 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include "registerAggregateFunctions.h"
// TODO include this last because of a broken roaring header. See the comment inside.
#include <AggregateFunctions/AggregateFunctionGroupBitmap.h>
namespace DB
{
struct Settings;
@ -18,9 +18,8 @@ namespace ErrorCodes
namespace
{
template <template <typename, typename> class AggregateFunctionTemplate, template <typename> class Data, typename... TArgs>
static IAggregateFunction * createWithIntegerType(const IDataType & argument_type, TArgs && ... args)
template <template <typename, typename> class AggregateFunctionTemplate, template <typename> typename Data, typename... TArgs>
static IAggregateFunction * createWithIntegerType(const IDataType & argument_type, TArgs &&... args)
{
WhichDataType which(argument_type);
if (which.idx == TypeIndex::UInt8) return new AggregateFunctionTemplate<UInt8, Data<UInt8>>(std::forward<TArgs>(args)...);
@ -34,8 +33,9 @@ namespace
return nullptr;
}
template <template <typename> class Data>
AggregateFunctionPtr createAggregateFunctionBitmap(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
template <template <typename> typename Data>
AggregateFunctionPtr createAggregateFunctionBitmap(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);
@ -57,12 +57,13 @@ namespace
}
// Additional aggregate functions to manipulate bitmaps.
template <template <typename, typename> class AggregateFunctionTemplate>
AggregateFunctionPtr
createAggregateFunctionBitmapL2(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
template <template <typename, typename> typename AggregateFunctionTemplate>
AggregateFunctionPtr createAggregateFunctionBitmapL2(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);
DataTypePtr argument_type_ptr = argument_types[0];
WhichDataType which(*argument_type_ptr);
if (which.idx != TypeIndex::AggregateFunction)
@ -70,11 +71,15 @@ namespace
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
/// groupBitmap needs to know about the data type that was used to create bitmaps.
/// We need to look inside the type of its argument to obtain it.
const DataTypeAggregateFunction & datatype_aggfunc = dynamic_cast<const DataTypeAggregateFunction &>(*argument_type_ptr);
AggregateFunctionPtr aggfunc = datatype_aggfunc.getFunction();
argument_type_ptr = aggfunc->getArgumentTypes()[0];
DataTypePtr nested_argument_type_ptr = aggfunc->getArgumentTypes()[0];
AggregateFunctionPtr res(createWithIntegerType<AggregateFunctionTemplate, AggregateFunctionGroupBitmapData>(
*argument_type_ptr, argument_type_ptr));
*nested_argument_type_ptr, argument_type_ptr));
if (!res)
throw Exception(
"Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,

View File

@ -9,15 +9,19 @@
// TODO include this last because of a broken roaring header. See the comment inside.
#include <AggregateFunctions/AggregateFunctionGroupBitmapData.h>
namespace DB
{
struct Settings;
/// Counts bitmap operation on numbers.
template <typename T, typename Data>
class AggregateFunctionBitmap final : public IAggregateFunctionDataHelper<Data, AggregateFunctionBitmap<T, Data>>
{
public:
AggregateFunctionBitmap(const DataTypePtr & type) : IAggregateFunctionDataHelper<Data, AggregateFunctionBitmap<T, Data>>({type}, {}) { }
AggregateFunctionBitmap(const DataTypePtr & type)
: IAggregateFunctionDataHelper<Data, AggregateFunctionBitmap<T, Data>>({type}, {})
{
}
String getName() const override { return Data::name(); }
@ -46,6 +50,7 @@ public:
};
/// This aggregate function takes the states of AggregateFunctionBitmap as its argument.
template <typename T, typename Data, typename Policy>
class AggregateFunctionBitmapL2 final : public IAggregateFunctionDataHelper<Data, AggregateFunctionBitmapL2<T, Data, Policy>>
{
@ -61,6 +66,11 @@ public:
bool allocatesMemoryInArena() const override { return false; }
DataTypePtr getStateType() const override
{
return this->argument_types.at(0);
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
Data & data_lhs = this->data(place);
@ -105,6 +115,7 @@ public:
}
};
template <typename Data>
class BitmapAndPolicy
{

View File

@ -1,9 +1,9 @@
#pragma once
#include <algorithm>
#include <boost/noncopyable.hpp>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <boost/noncopyable.hpp>
#include <Common/HashTable/SmallTable.h>
#include <Common/PODArray.h>
@ -14,9 +14,9 @@
#include <roaring.hh>
#include <roaring64map.hh>
namespace DB
{
struct Settings;
enum BitmapKind
{
@ -24,6 +24,7 @@ enum BitmapKind
Bitmap = 1
};
/**
* For a small number of values - an array of fixed size "on the stack".
* For large, roaring bitmap is allocated.

View File

@ -4,7 +4,6 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -73,7 +72,8 @@ inline AggregateFunctionPtr createAggregateFunctionGroupUniqArrayImpl(const std:
}
AggregateFunctionPtr createAggregateFunctionGroupUniqArray(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
AggregateFunctionPtr createAggregateFunctionGroupUniqArray(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertUnary(name, argument_types);

View File

@ -4,7 +4,7 @@
#include <AggregateFunctions/Helpers.h>
#include <Common/FieldVisitors.h>
#include "registerAggregateFunctions.h"
namespace DB
{

View File

@ -1,12 +1,10 @@
#include <AggregateFunctions/AggregateFunctionIf.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include "registerAggregateFunctions.h"
#include "AggregateFunctionNull.h"
namespace DB
{
struct Settings;
namespace ErrorCodes
{

View File

@ -11,7 +11,6 @@
#include "AggregateFunctionFactory.h"
#include "FactoryHelpers.h"
#include "Helpers.h"
#include "registerAggregateFunctions.h"
namespace DB
@ -25,13 +24,14 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
using FuncLinearRegression = AggregateFunctionMLMethod<LinearModelData, NameLinearRegression>;
using FuncLogisticRegression = AggregateFunctionMLMethod<LinearModelData, NameLogisticRegression>;
template <class Method>
AggregateFunctionPtr
createAggregateFunctionMLMethod(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
template <typename Method>
AggregateFunctionPtr createAggregateFunctionMLMethod(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
if (parameters.size() > 4)
throw Exception(

View File

@ -1,7 +1,6 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionMannWhitney.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
#include <AggregateFunctions/Helpers.h>
@ -17,7 +16,8 @@ struct Settings;
namespace
{
AggregateFunctionPtr createAggregateFunctionMannWhitneyUTest(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
AggregateFunctionPtr createAggregateFunctionMannWhitneyUTest(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertBinary(name, argument_types);

View File

@ -1,7 +1,6 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/HelpersMinMaxAny.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB
@ -11,12 +10,14 @@ struct Settings;
namespace
{
AggregateFunctionPtr createAggregateFunctionMax(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
AggregateFunctionPtr createAggregateFunctionMax(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMaxData>(name, argument_types, parameters, settings));
}
AggregateFunctionPtr createAggregateFunctionArgMax(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
AggregateFunctionPtr createAggregateFunctionArgMax(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionArgMinMax<AggregateFunctionMaxData>(name, argument_types, parameters, settings));
}

View File

@ -2,7 +2,6 @@
#include <AggregateFunctions/AggregateFunctionMaxIntersections.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include "registerAggregateFunctions.h"
namespace DB

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