mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge branch 'master' into deprecate_ordinary_database
This commit is contained in:
commit
7c37079a96
12
.gitmodules
vendored
12
.gitmodules
vendored
@ -86,9 +86,6 @@
|
||||
[submodule "contrib/h3"]
|
||||
path = contrib/h3
|
||||
url = https://github.com/ClickHouse/h3
|
||||
[submodule "contrib/hyperscan"]
|
||||
path = contrib/hyperscan
|
||||
url = https://github.com/ClickHouse/hyperscan.git
|
||||
[submodule "contrib/libunwind"]
|
||||
path = contrib/libunwind
|
||||
url = https://github.com/ClickHouse/libunwind.git
|
||||
@ -268,3 +265,12 @@
|
||||
[submodule "contrib/hashidsxx"]
|
||||
path = contrib/hashidsxx
|
||||
url = https://github.com/schoentoon/hashidsxx.git
|
||||
[submodule "contrib/vectorscan"]
|
||||
path = contrib/vectorscan
|
||||
url = https://github.com/VectorCamp/vectorscan.git
|
||||
[submodule "contrib/liburing"]
|
||||
path = contrib/liburing
|
||||
url = https://github.com/axboe/liburing.git
|
||||
[submodule "contrib/base-x"]
|
||||
path = contrib/base-x
|
||||
url = https://github.com/ClickHouse/base-x.git
|
||||
|
@ -1,180 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
# script to run query to databases
|
||||
|
||||
function usage()
|
||||
{
|
||||
cat <<EOF
|
||||
usage: $0 options
|
||||
|
||||
This script run benhmark for database
|
||||
|
||||
OPTIONS:
|
||||
-c config file where some script variables are defined
|
||||
-n table name
|
||||
|
||||
-h Show this message
|
||||
-t how many times execute each query. default is '3'
|
||||
-q query file
|
||||
-e expect file
|
||||
-s /etc/init.d/service
|
||||
-p table name pattern to be replaced to name. default is 'hits_10m'
|
||||
EOF
|
||||
}
|
||||
|
||||
TIMES=3
|
||||
table_name_pattern=hits_10m
|
||||
|
||||
while getopts “c:ht:n:q:e:s:r” OPTION
|
||||
do
|
||||
case $OPTION in
|
||||
c)
|
||||
source $OPTARG
|
||||
;;
|
||||
?)
|
||||
;;
|
||||
esac
|
||||
done
|
||||
|
||||
OPTIND=1
|
||||
|
||||
while getopts “c:ht:n:q:e:s:r” OPTION
|
||||
do
|
||||
case $OPTION in
|
||||
h)
|
||||
usage
|
||||
exit 0
|
||||
;;
|
||||
t)
|
||||
TIMES=$OPTARG
|
||||
;;
|
||||
n)
|
||||
table_name=$OPTARG
|
||||
;;
|
||||
q)
|
||||
test_file=$OPTARG
|
||||
;;
|
||||
e)
|
||||
expect_file=$OPTARG
|
||||
;;
|
||||
s)
|
||||
etc_init_d_service=$OPTARG
|
||||
;;
|
||||
p)
|
||||
table_name_pattern=$OPTARG
|
||||
;;
|
||||
c)
|
||||
;;
|
||||
r)
|
||||
restart_server_each_query=1
|
||||
;;
|
||||
?)
|
||||
usage
|
||||
exit 0
|
||||
;;
|
||||
esac
|
||||
done
|
||||
|
||||
if [[ ! -f $expect_file ]]; then
|
||||
echo "Not found: expect file"
|
||||
exit 1
|
||||
fi
|
||||
if [[ ! -f $test_file ]]; then
|
||||
echo "Not found: test file"
|
||||
exit 1
|
||||
fi
|
||||
|
||||
if [[ ! -f $etc_init_d_service ]]; then
|
||||
echo "Not found: /etc/init.d/service with path=$etc_init_d_service"
|
||||
use_service=0
|
||||
else
|
||||
use_service=1
|
||||
fi
|
||||
|
||||
if [[ "$table_name_pattern" == "" ]]; then
|
||||
echo "Empty table_name_pattern"
|
||||
exit 1
|
||||
fi
|
||||
if [[ "$table_name" == "" ]]; then
|
||||
echo "Empty table_name"
|
||||
exit 1
|
||||
fi
|
||||
|
||||
function execute()
|
||||
{
|
||||
queries=("${@}")
|
||||
queries_count=${#queries[@]}
|
||||
|
||||
if [ -z $TIMES ]; then
|
||||
TIMES=1
|
||||
fi
|
||||
|
||||
index=0
|
||||
while [ "$index" -lt "$queries_count" ]; do
|
||||
query=${queries[$index]}
|
||||
|
||||
if [[ $query == "" ]]; then
|
||||
let "index = $index + 1"
|
||||
continue
|
||||
fi
|
||||
|
||||
comment_re='--.*'
|
||||
if [[ $query =~ $comment_re ]]; then
|
||||
echo "$query"
|
||||
echo
|
||||
else
|
||||
sync
|
||||
sudo sh -c "echo 3 > /proc/sys/vm/drop_caches"
|
||||
|
||||
if [[ "$restart_server_each_query" == "1" && "$use_service" == "1" ]]; then
|
||||
echo "restart server: $etc_init_d_service restart"
|
||||
sudo $etc_init_d_service restart
|
||||
fi
|
||||
|
||||
for i in $(seq $TIMES)
|
||||
do
|
||||
if [[ -f $etc_init_d_service && "$use_service" == "1" ]]; then
|
||||
sudo $etc_init_d_service status
|
||||
server_status=$?
|
||||
expect -f $expect_file ""
|
||||
|
||||
if [[ "$?" != "0" || $server_status != "0" ]]; then
|
||||
echo "restart server: $etc_init_d_service restart"
|
||||
sudo $etc_init_d_service restart
|
||||
fi
|
||||
|
||||
#wait until can connect to server
|
||||
restart_timer=0
|
||||
restart_limit=60
|
||||
expect -f $expect_file "" &> /dev/null
|
||||
while [ "$?" != "0" ]; do
|
||||
echo "waiting"
|
||||
sleep 1
|
||||
let "restart_timer = $restart_timer + 1"
|
||||
if (( $restart_limit < $restart_timer )); then
|
||||
sudo $etc_init_d_service restart
|
||||
restart_timer=0
|
||||
fi
|
||||
expect -f $expect_file "" &> /dev/null
|
||||
done
|
||||
fi
|
||||
|
||||
echo
|
||||
echo "times: $i"
|
||||
|
||||
echo "query:" "$query"
|
||||
expect -f $expect_file "$query"
|
||||
|
||||
done
|
||||
fi
|
||||
|
||||
let "index = $index + 1"
|
||||
done
|
||||
}
|
||||
|
||||
temp_test_file=temp_queries_$table_name
|
||||
cat $test_file | sed s/$table_name_pattern/$table_name/g > $temp_test_file
|
||||
mapfile -t test_queries < $temp_test_file
|
||||
|
||||
echo "start time: $(date)"
|
||||
time execute "${test_queries[@]}"
|
||||
echo "stop time: $(date)"
|
@ -1,3 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
table=hits_10m; time clickhouse-client --max_bytes_before_external_sort=30000000000 --query="SELECT toInt64(WatchID), JavaEnable, Title, GoodEvent, (EventTime < toDateTime('1971-01-01 00:00:00') ? toDateTime('1971-01-01 00:00:01') : EventTime), (EventDate < toDate('1971-01-01') ? toDate('1971-01-01') : EventDate), CounterID, ClientIP, RegionID, toInt64(UserID), CounterClass, OS, UserAgent, URL, Referer, Refresh, RefererCategoryID, RefererRegionID, URLCategoryID, URLRegionID, ResolutionWidth, ResolutionHeight, ResolutionDepth, FlashMajor, FlashMinor, FlashMinor2, NetMajor, NetMinor, UserAgentMajor, UserAgentMinor, CookieEnable, JavascriptEnable, IsMobile, MobilePhone, MobilePhoneModel, Params, IPNetworkID, TraficSourceID, SearchEngineID, SearchPhrase, AdvEngineID, IsArtifical, WindowClientWidth, WindowClientHeight, ClientTimeZone, (ClientEventTime < toDateTime('1971-01-01 00:00:01') ? toDateTime('1971-01-01 00:00:01') : ClientEventTime), SilverlightVersion1, SilverlightVersion2, SilverlightVersion3, SilverlightVersion4, PageCharset, CodeVersion, IsLink, IsDownload, IsNotBounce, toInt64(FUniqID), OriginalURL, HID, IsOldCounter, IsEvent, IsParameter, DontCountHits, WithHash, HitColor, (LocalEventTime < toDateTime('1971-01-01 00:00:01') ? toDateTime('1971-01-01 00:00:01') : LocalEventTime), Age, Sex, Income, Interests, Robotness, RemoteIP, WindowName, OpenerName, HistoryLength, BrowserLanguage, BrowserCountry, SocialNetwork, SocialAction, HTTPError, SendTiming, DNSTiming, ConnectTiming, ResponseStartTiming, ResponseEndTiming, FetchTiming, SocialSourceNetworkID, SocialSourcePage, ParamPrice, ParamOrderID, ParamCurrency, ParamCurrencyID, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID, toInt64(RefererHash), toInt64(URLHash), CLID FROM $table ORDER BY rand()" | corrector_utf8 > /opt/dumps/${table}_corrected.tsv
|
@ -1,107 +1,25 @@
|
||||
#!/bin/bash -e
|
||||
|
||||
if [[ -n $1 ]]; then
|
||||
SCALE=$1
|
||||
else
|
||||
SCALE=100
|
||||
fi
|
||||
|
||||
TABLE="hits_${SCALE}m_obfuscated"
|
||||
DATASET="${TABLE}_v1.tar.xz"
|
||||
QUERIES_FILE="queries.sql"
|
||||
TRIES=3
|
||||
|
||||
# Note: on older Ubuntu versions, 'axel' does not support IPv6. If you are using IPv6-only servers on very old Ubuntu, just don't install 'axel'.
|
||||
mkdir -p clickhouse-benchmark
|
||||
pushd clickhouse-benchmark
|
||||
|
||||
FASTER_DOWNLOAD=wget
|
||||
if command -v axel >/dev/null; then
|
||||
FASTER_DOWNLOAD=axel
|
||||
else
|
||||
echo "It's recommended to install 'axel' for faster downloads."
|
||||
# Download the binary
|
||||
if [[ ! -x clickhouse ]]; then
|
||||
curl https://clickhouse.com/ | sh
|
||||
fi
|
||||
|
||||
if command -v pixz >/dev/null; then
|
||||
TAR_PARAMS='-Ipixz'
|
||||
else
|
||||
echo "It's recommended to install 'pixz' for faster decompression of the dataset."
|
||||
fi
|
||||
|
||||
mkdir -p clickhouse-benchmark-$SCALE
|
||||
pushd clickhouse-benchmark-$SCALE
|
||||
|
||||
OS=$(uname -s)
|
||||
ARCH=$(uname -m)
|
||||
|
||||
DIR=
|
||||
|
||||
if [ "${OS}" = "Linux" ]
|
||||
then
|
||||
if [ "${ARCH}" = "x86_64" ]
|
||||
then
|
||||
DIR="amd64"
|
||||
elif [ "${ARCH}" = "aarch64" ]
|
||||
then
|
||||
DIR="aarch64"
|
||||
elif [ "${ARCH}" = "powerpc64le" ]
|
||||
then
|
||||
DIR="powerpc64le"
|
||||
fi
|
||||
elif [ "${OS}" = "FreeBSD" ]
|
||||
then
|
||||
if [ "${ARCH}" = "x86_64" ]
|
||||
then
|
||||
DIR="freebsd"
|
||||
elif [ "${ARCH}" = "aarch64" ]
|
||||
then
|
||||
DIR="freebsd-aarch64"
|
||||
elif [ "${ARCH}" = "powerpc64le" ]
|
||||
then
|
||||
DIR="freebsd-powerpc64le"
|
||||
fi
|
||||
elif [ "${OS}" = "Darwin" ]
|
||||
then
|
||||
if [ "${ARCH}" = "x86_64" ]
|
||||
then
|
||||
DIR="macos"
|
||||
elif [ "${ARCH}" = "aarch64" -o "${ARCH}" = "arm64" ]
|
||||
then
|
||||
DIR="macos-aarch64"
|
||||
fi
|
||||
fi
|
||||
|
||||
if [ -z "${DIR}" ]
|
||||
then
|
||||
echo "The '${OS}' operating system with the '${ARCH}' architecture is not supported."
|
||||
exit 1
|
||||
fi
|
||||
|
||||
URL="https://builds.clickhouse.com/master/${DIR}/clickhouse"
|
||||
echo
|
||||
echo "Will download ${URL}"
|
||||
echo
|
||||
curl -O "${URL}" && chmod a+x clickhouse || exit 1
|
||||
echo
|
||||
echo "Successfully downloaded the ClickHouse binary"
|
||||
|
||||
chmod a+x clickhouse
|
||||
|
||||
if [[ ! -f $QUERIES_FILE ]]; then
|
||||
wget "https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/clickhouse/$QUERIES_FILE"
|
||||
fi
|
||||
|
||||
if [[ ! -d data ]]; then
|
||||
if [[ ! -f $DATASET ]]; then
|
||||
$FASTER_DOWNLOAD "https://datasets.clickhouse.com/hits/partitions/$DATASET"
|
||||
fi
|
||||
|
||||
tar $TAR_PARAMS --strip-components=1 --directory=. -x -v -f $DATASET
|
||||
fi
|
||||
|
||||
uptime
|
||||
|
||||
echo "Starting clickhouse-server"
|
||||
|
||||
./clickhouse server > server.log 2>&1 &
|
||||
./clickhouse server >/dev/null 2>&1 &
|
||||
PID=$!
|
||||
|
||||
function finish {
|
||||
@ -114,15 +32,36 @@ echo "Waiting for clickhouse-server to start"
|
||||
|
||||
for i in {1..30}; do
|
||||
sleep 1
|
||||
./clickhouse client --query "SELECT 'The dataset size is: ', count() FROM $TABLE" 2>/dev/null && break || echo '.'
|
||||
./clickhouse client --query "SELECT 'Ok.'" 2>/dev/null && break || echo -n '.'
|
||||
if [[ $i == 30 ]]; then exit 1; fi
|
||||
done
|
||||
|
||||
if [[ $(./clickhouse client --query "EXISTS hits") == '1' && $(./clickhouse client --query "SELECT count() FROM hits") == '100000000' ]]; then
|
||||
echo "Dataset already downloaded"
|
||||
else
|
||||
echo "Will download the dataset"
|
||||
./clickhouse client --receive_timeout 1000 --max_insert_threads $(nproc || 4) --progress --query "
|
||||
CREATE OR REPLACE TABLE hits ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime)
|
||||
AS SELECT * FROM url('https://datasets.clickhouse.com/hits/native/hits_100m_obfuscated_{0..255}.native.zst')"
|
||||
|
||||
./clickhouse client --query "SELECT 'The dataset size is: ', count() FROM hits"
|
||||
fi
|
||||
|
||||
if [[ $(./clickhouse client --query "SELECT count() FROM system.parts WHERE table = 'hits' AND database = 'default' AND active") == '1' ]]; then
|
||||
echo "Dataset already prepared"
|
||||
else
|
||||
echo "Will prepare the dataset"
|
||||
./clickhouse client --receive_timeout 1000 --query "OPTIMIZE TABLE hits FINAL"
|
||||
fi
|
||||
|
||||
echo
|
||||
echo "Will perform benchmark. Results:"
|
||||
echo
|
||||
|
||||
cat "$QUERIES_FILE" | sed "s/{table}/${TABLE}/g" | while read query; do
|
||||
>result.csv
|
||||
QUERY_NUM=1
|
||||
|
||||
cat "$QUERIES_FILE" | sed "s/{table}/hits/g" | while read query; do
|
||||
sync
|
||||
if [ "${OS}" = "Darwin" ]
|
||||
then
|
||||
@ -133,11 +72,15 @@ cat "$QUERIES_FILE" | sed "s/{table}/${TABLE}/g" | while read query; do
|
||||
|
||||
echo -n "["
|
||||
for i in $(seq 1 $TRIES); do
|
||||
RES=$(./clickhouse client --max_memory_usage 100G --time --format=Null --query="$query" 2>&1 ||:)
|
||||
RES=$(./clickhouse client --time --format=Null --query="$query" 2>&1 ||:)
|
||||
[[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null"
|
||||
[[ "$i" != $TRIES ]] && echo -n ", "
|
||||
|
||||
echo "${QUERY_NUM},${i},${RES}" >> result.csv
|
||||
done
|
||||
echo "],"
|
||||
|
||||
QUERY_NUM=$((QUERY_NUM + 1))
|
||||
done
|
||||
|
||||
|
||||
@ -145,22 +88,23 @@ echo
|
||||
echo "Benchmark complete. System info:"
|
||||
echo
|
||||
|
||||
touch {cpu_model,cpu,df,memory,memory_total,blk,mdstat,instance}.txt
|
||||
|
||||
if [ "${OS}" = "Darwin" ]
|
||||
then
|
||||
echo '----Version, build id-----------'
|
||||
./clickhouse local --query "SELECT format('Version: {}', version())"
|
||||
sw_vers | grep BuildVersion
|
||||
./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw
|
||||
./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))"
|
||||
echo '----CPU-------------------------'
|
||||
sysctl hw.model
|
||||
sysctl -a | grep -E 'hw.activecpu|hw.memsize|hw.byteorder|cachesize'
|
||||
sysctl hw.model | tee cpu_model.txt
|
||||
sysctl -a | grep -E 'hw.activecpu|hw.memsize|hw.byteorder|cachesize' | tee cpu.txt
|
||||
echo '----Disk Free and Total--------'
|
||||
df -h .
|
||||
df -h . | tee df.txt
|
||||
echo '----Memory Free and Total-------'
|
||||
vm_stat
|
||||
vm_stat | tee memory.txt
|
||||
echo '----Physical Memory Amount------'
|
||||
ls -l /var/vm
|
||||
ls -l /var/vm | tee memory_total.txt
|
||||
echo '--------------------------------'
|
||||
else
|
||||
echo '----Version, build id-----------'
|
||||
@ -168,22 +112,103 @@ else
|
||||
./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw
|
||||
./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))"
|
||||
echo '----CPU-------------------------'
|
||||
cat /proc/cpuinfo | grep -i -F 'model name' | uniq
|
||||
lscpu
|
||||
cat /proc/cpuinfo | grep -i -F 'model name' | uniq | tee cpu_model.txt
|
||||
lscpu | tee cpu.txt
|
||||
echo '----Block Devices---------------'
|
||||
lsblk
|
||||
lsblk | tee blk.txt
|
||||
echo '----Disk Free and Total--------'
|
||||
df -h .
|
||||
df -h . | tee df.txt
|
||||
echo '----Memory Free and Total-------'
|
||||
free -h
|
||||
free -h | tee memory.txt
|
||||
echo '----Physical Memory Amount------'
|
||||
cat /proc/meminfo | grep MemTotal
|
||||
cat /proc/meminfo | grep MemTotal | tee memory_total.txt
|
||||
echo '----RAID Info-------------------'
|
||||
cat /proc/mdstat
|
||||
#echo '----PCI-------------------------'
|
||||
#lspci
|
||||
#echo '----All Hardware Info-----------'
|
||||
#lshw
|
||||
cat /proc/mdstat| tee mdstat.txt
|
||||
echo '--------------------------------'
|
||||
fi
|
||||
echo
|
||||
|
||||
echo "Instance type from IMDS (if available):"
|
||||
curl -s --connect-timeout 1 'http://169.254.169.254/latest/meta-data/instance-type' | tee instance.txt
|
||||
echo
|
||||
|
||||
echo "Uploading the results (if possible)"
|
||||
|
||||
UUID=$(./clickhouse local --query "SELECT generateUUIDv4()")
|
||||
|
||||
./clickhouse local --query "
|
||||
SELECT
|
||||
'${UUID}' AS run_id,
|
||||
version() AS version,
|
||||
now() AS test_time,
|
||||
(SELECT value FROM system.settings WHERE name = 'max_threads') AS threads,
|
||||
filesystemCapacity() AS fs_capacity,
|
||||
filesystemAvailable() AS fs_available,
|
||||
file('cpu_model.txt') AS cpu_model,
|
||||
file('cpu.txt') AS cpu,
|
||||
file('df.txt') AS df,
|
||||
file('memory.txt') AS memory,
|
||||
file('memory_total.txt') AS memory_total,
|
||||
file('blk.txt') AS blk,
|
||||
file('mdstat.txt') AS mdstat,
|
||||
file('instance.txt') AS instance
|
||||
" | tee meta.tsv | ./clickhouse client --host play.clickhouse.com --secure --user benchmark --query "
|
||||
INSERT INTO benchmark_runs
|
||||
(run_id, version, test_time, threads, fs_capacity, fs_available, cpu_model, cpu, df, memory, memory_total, blk, mdstat, instance)
|
||||
FORMAT TSV" || echo "Cannot upload results."
|
||||
|
||||
./clickhouse local --query "
|
||||
SELECT
|
||||
'${UUID}' AS run_id,
|
||||
c1 AS query_num,
|
||||
c2 AS try_num,
|
||||
c3 AS time
|
||||
FROM file('result.csv')
|
||||
" | tee results.tsv | ./clickhouse client --host play.clickhouse.com --secure --user benchmark --query "
|
||||
INSERT INTO benchmark_results
|
||||
(run_id, query_num, try_num, time)
|
||||
FORMAT TSV" || echo "Cannot upload results. Please send the output to feedback@clickhouse.com"
|
||||
|
||||
<<////
|
||||
|
||||
Server Setup:
|
||||
|
||||
CREATE TABLE benchmark_runs
|
||||
(
|
||||
run_id UUID,
|
||||
version String,
|
||||
test_time DateTime,
|
||||
threads String,
|
||||
fs_capacity UInt64,
|
||||
fs_available UInt64,
|
||||
cpu_model String,
|
||||
cpu String,
|
||||
df String,
|
||||
memory String,
|
||||
memory_total String,
|
||||
blk String,
|
||||
mdstat String,
|
||||
instance String
|
||||
) ENGINE = ReplicatedMergeTree ORDER BY run_id;
|
||||
|
||||
CREATE TABLE benchmark_results
|
||||
(
|
||||
run_id UUID,
|
||||
query_num UInt8,
|
||||
try_num UInt8,
|
||||
time Decimal32(3)
|
||||
) ENGINE = ReplicatedMergeTree ORDER BY (run_id, query_num, try_num);
|
||||
|
||||
CREATE USER benchmark IDENTIFIED WITH no_password SETTINGS max_rows_to_read = 1, max_result_rows = 1, max_execution_time = 1;
|
||||
|
||||
CREATE QUOTA benchmark
|
||||
KEYED BY ip_address
|
||||
FOR RANDOMIZED INTERVAL 1 MINUTE MAX query_inserts = 4, written_bytes = 100000,
|
||||
FOR RANDOMIZED INTERVAL 1 HOUR MAX query_inserts = 10, written_bytes = 500000,
|
||||
FOR RANDOMIZED INTERVAL 1 DAY MAX query_inserts = 50, written_bytes = 2000000
|
||||
TO benchmark;
|
||||
|
||||
GRANT INSERT ON benchmark_runs TO benchmark;
|
||||
GRANT INSERT ON benchmark_results TO benchmark;
|
||||
|
||||
////
|
||||
|
3
contrib/CMakeLists.txt
vendored
3
contrib/CMakeLists.txt
vendored
@ -58,7 +58,7 @@ add_contrib (boost-cmake boost)
|
||||
add_contrib (cctz-cmake cctz)
|
||||
add_contrib (consistent-hashing)
|
||||
add_contrib (dragonbox-cmake dragonbox)
|
||||
add_contrib (hyperscan-cmake hyperscan)
|
||||
add_contrib (vectorscan-cmake vectorscan)
|
||||
add_contrib (jemalloc-cmake jemalloc)
|
||||
add_contrib (libcpuid-cmake libcpuid)
|
||||
add_contrib (libdivide)
|
||||
@ -155,6 +155,7 @@ endif()
|
||||
|
||||
add_contrib (sqlite-cmake sqlite-amalgamation)
|
||||
add_contrib (s2geometry-cmake s2geometry)
|
||||
add_contrib (base-x-cmake base-x)
|
||||
|
||||
# Put all targets defined here and in subdirectories under "contrib/<immediate-subdir>" folders in GUI-based IDEs.
|
||||
# Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear
|
||||
|
1
contrib/base-x
vendored
Submodule
1
contrib/base-x
vendored
Submodule
@ -0,0 +1 @@
|
||||
Subproject commit a85f98fb4ed52c2f4029a4b6ac1ef0bafdfc56f5
|
28
contrib/base-x-cmake/CMakeLists.txt
Normal file
28
contrib/base-x-cmake/CMakeLists.txt
Normal file
@ -0,0 +1,28 @@
|
||||
option (ENABLE_BASEX "Enable base-x" ${ENABLE_LIBRARIES})
|
||||
|
||||
if (NOT ENABLE_BASEX)
|
||||
message(STATUS "Not using base-x")
|
||||
return()
|
||||
endif()
|
||||
|
||||
set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/base-x")
|
||||
|
||||
set (SRCS
|
||||
${LIBRARY_DIR}/base_x.hh
|
||||
${LIBRARY_DIR}/uinteger_t.hh
|
||||
)
|
||||
|
||||
add_library(_base-x INTERFACE)
|
||||
target_include_directories(_base-x SYSTEM BEFORE INTERFACE "${ClickHouse_SOURCE_DIR}/contrib/base-x")
|
||||
|
||||
if (XCODE OR XCODE_VERSION)
|
||||
# https://gitlab.kitware.com/cmake/cmake/issues/17457
|
||||
# Some native build systems may not like targets that have only object files, so consider adding at least one real source file
|
||||
# This applies to Xcode.
|
||||
if (NOT EXISTS "${CMAKE_CURRENT_BINARY_DIR}/dummy.c")
|
||||
file(WRITE "${CMAKE_CURRENT_BINARY_DIR}/dummy.c" "")
|
||||
endif ()
|
||||
target_sources(_base-x PRIVATE "${CMAKE_CURRENT_BINARY_DIR}/dummy.c")
|
||||
endif ()
|
||||
|
||||
add_library(ch_contrib::base-x ALIAS _base-x)
|
1
contrib/hyperscan
vendored
1
contrib/hyperscan
vendored
@ -1 +0,0 @@
|
||||
Subproject commit 5edc68c5ac68d2d4f876159e9ee84def6d3dc87c
|
1
contrib/vectorscan
vendored
Submodule
1
contrib/vectorscan
vendored
Submodule
@ -0,0 +1 @@
|
||||
Subproject commit 73695e419c27af7fe2a099c7aa57931cc02aea5d
|
@ -1,54 +1,67 @@
|
||||
if (HAVE_SSSE3)
|
||||
option (ENABLE_HYPERSCAN "Enable hyperscan library" ${ENABLE_LIBRARIES})
|
||||
elseif(ENABLE_HYPERSCAN)
|
||||
message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use hyperscan without SSSE3")
|
||||
set (ENABLE_HYPERSCAN OFF)
|
||||
endif ()
|
||||
# We use vectorscan, a portable and API/ABI-compatible drop-in replacement for hyperscan.
|
||||
|
||||
if (NOT ENABLE_HYPERSCAN)
|
||||
message (STATUS "Not using hyperscan")
|
||||
if (ARCH_AMD64)
|
||||
option (ENABLE_VECTORSCAN "Enable vectorscan library" ${ENABLE_LIBRARIES})
|
||||
endif()
|
||||
|
||||
# TODO: vectorscan supports ARM yet some tests involving cyrillic letters fail (PR #38171) ... needs further investigation
|
||||
|
||||
# TODO PPC should generally work but needs manual generation of ppc/config.h file on a PPC machine
|
||||
|
||||
if (NOT ENABLE_VECTORSCAN)
|
||||
message (STATUS "Not using vectorscan")
|
||||
return()
|
||||
endif()
|
||||
|
||||
set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/hyperscan")
|
||||
set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/vectorscan")
|
||||
|
||||
# Gobble up all c/cpp files in vectorscan/src/, omit *dump*.c/cpp files as we don't use the dump feature (see x86/config.h)
|
||||
set (SRCS
|
||||
"${LIBRARY_DIR}/src/alloc.c"
|
||||
"${LIBRARY_DIR}/src/crc32.c"
|
||||
"${LIBRARY_DIR}/src/database.c"
|
||||
# "${LIBRARY_DIR}/src/dispatcher.c" # the linker's wrath be upon those who include dispatcher.c.
|
||||
"${LIBRARY_DIR}/src/grey.cpp"
|
||||
"${LIBRARY_DIR}/src/hs.cpp"
|
||||
"${LIBRARY_DIR}/src/hs_valid_platform.c"
|
||||
"${LIBRARY_DIR}/src/hs_version.c"
|
||||
"${LIBRARY_DIR}/src/runtime.c"
|
||||
"${LIBRARY_DIR}/src/scratch.c"
|
||||
"${LIBRARY_DIR}/src/stream_compress.c"
|
||||
|
||||
"${LIBRARY_DIR}/src/compiler/asserts.cpp"
|
||||
"${LIBRARY_DIR}/src/compiler/compiler.cpp"
|
||||
"${LIBRARY_DIR}/src/compiler/error.cpp"
|
||||
"${LIBRARY_DIR}/src/crc32.c"
|
||||
"${LIBRARY_DIR}/src/database.c"
|
||||
|
||||
"${LIBRARY_DIR}/src/fdr/engine_description.cpp"
|
||||
"${LIBRARY_DIR}/src/fdr/fdr_compile_util.cpp"
|
||||
"${LIBRARY_DIR}/src/fdr/fdr.c"
|
||||
"${LIBRARY_DIR}/src/fdr/fdr_compile.cpp"
|
||||
"${LIBRARY_DIR}/src/fdr/fdr_compile_util.cpp"
|
||||
"${LIBRARY_DIR}/src/fdr/fdr_confirm_compile.cpp"
|
||||
"${LIBRARY_DIR}/src/fdr/fdr_engine_description.cpp"
|
||||
"${LIBRARY_DIR}/src/fdr/fdr.c"
|
||||
"${LIBRARY_DIR}/src/fdr/flood_compile.cpp"
|
||||
"${LIBRARY_DIR}/src/fdr/teddy.c"
|
||||
"${LIBRARY_DIR}/src/fdr/teddy_avx2.c"
|
||||
"${LIBRARY_DIR}/src/fdr/teddy_compile.cpp"
|
||||
"${LIBRARY_DIR}/src/fdr/teddy_engine_description.cpp"
|
||||
"${LIBRARY_DIR}/src/fdr/teddy.c"
|
||||
"${LIBRARY_DIR}/src/grey.cpp"
|
||||
"${LIBRARY_DIR}/src/hs_valid_platform.c"
|
||||
"${LIBRARY_DIR}/src/hs_version.c"
|
||||
"${LIBRARY_DIR}/src/hs.cpp"
|
||||
|
||||
"${LIBRARY_DIR}/src/hwlm/hwlm.c"
|
||||
"${LIBRARY_DIR}/src/hwlm/hwlm_build.cpp"
|
||||
"${LIBRARY_DIR}/src/hwlm/hwlm_literal.cpp"
|
||||
"${LIBRARY_DIR}/src/hwlm/hwlm.c"
|
||||
"${LIBRARY_DIR}/src/hwlm/noodle_build.cpp"
|
||||
"${LIBRARY_DIR}/src/hwlm/noodle_engine.c"
|
||||
"${LIBRARY_DIR}/src/nfa/accel_dfa_build_strat.cpp"
|
||||
"${LIBRARY_DIR}/src/hwlm/noodle_engine.cpp"
|
||||
|
||||
"${LIBRARY_DIR}/src/nfa/accel.c"
|
||||
"${LIBRARY_DIR}/src/nfa/accel_dfa_build_strat.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/accelcompile.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/castle.c"
|
||||
"${LIBRARY_DIR}/src/nfa/castlecompile.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/dfa_build_strat.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/dfa_min.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/gough.c"
|
||||
"${LIBRARY_DIR}/src/nfa/goughcompile.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/goughcompile_accel.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/goughcompile_reg.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/goughcompile.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/lbr.c"
|
||||
"${LIBRARY_DIR}/src/nfa/limex_64.c"
|
||||
"${LIBRARY_DIR}/src/nfa/limex_accel.c"
|
||||
@ -59,28 +72,32 @@ set (SRCS
|
||||
"${LIBRARY_DIR}/src/nfa/limex_simd384.c"
|
||||
"${LIBRARY_DIR}/src/nfa/limex_simd512.c"
|
||||
"${LIBRARY_DIR}/src/nfa/mcclellan.c"
|
||||
"${LIBRARY_DIR}/src/nfa/mcclellancompile_util.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/mcclellancompile.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/mcclellancompile_util.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/mcsheng.c"
|
||||
"${LIBRARY_DIR}/src/nfa/mcsheng_compile.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/mcsheng_data.c"
|
||||
"${LIBRARY_DIR}/src/nfa/mcsheng.c"
|
||||
"${LIBRARY_DIR}/src/nfa/mpv.c"
|
||||
"${LIBRARY_DIR}/src/nfa/mpvcompile.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/nfa_api_dispatch.c"
|
||||
"${LIBRARY_DIR}/src/nfa/nfa_build_util.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/rdfa.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/rdfa_graph.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/rdfa_merge.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/rdfa.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/repeat.c"
|
||||
"${LIBRARY_DIR}/src/nfa/repeatcompile.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/sheng.c"
|
||||
"${LIBRARY_DIR}/src/nfa/shengcompile.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/shufti.c"
|
||||
"${LIBRARY_DIR}/src/nfa/shufti.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/shufticompile.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/tamarama.c"
|
||||
"${LIBRARY_DIR}/src/nfa/tamaramacompile.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/truffle.c"
|
||||
"${LIBRARY_DIR}/src/nfa/truffle.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/trufflecompile.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/vermicelli_simd.cpp"
|
||||
"${LIBRARY_DIR}/src/nfa/vermicellicompile.cpp"
|
||||
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_anchored_acyclic.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_anchored_dots.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_asserts.cpp"
|
||||
@ -100,8 +117,8 @@ set (SRCS
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_holder.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_is_equal.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_lbr.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_limex_accel.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_limex.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_limex_accel.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_literal_analysis.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_literal_component.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_literal_decorated.cpp"
|
||||
@ -112,17 +129,17 @@ set (SRCS
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_prune.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_puff.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_redundancy.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_region_redundancy.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_region.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_region_redundancy.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_repeat.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_reports.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_restructuring.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_revacc.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_sep.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_small_literal_set.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_som.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_som_add_redundancy.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_som_util.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_som.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_split.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_squash.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_stop.cpp"
|
||||
@ -132,10 +149,8 @@ set (SRCS
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_vacuous.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_violet.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng_width.cpp"
|
||||
"${LIBRARY_DIR}/src/nfagraph/ng.cpp"
|
||||
|
||||
"${LIBRARY_DIR}/src/parser/AsciiComponentClass.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/buildstate.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/check_refs.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/Component.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/ComponentAlternation.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/ComponentAssertion.cpp"
|
||||
@ -145,31 +160,34 @@ set (SRCS
|
||||
"${LIBRARY_DIR}/src/parser/ComponentByte.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/ComponentClass.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/ComponentCondReference.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/ComponentEmpty.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/ComponentEUS.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/ComponentEmpty.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/ComponentRepeat.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/ComponentSequence.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/ComponentVisitor.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/ComponentWordBoundary.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/ConstComponentVisitor.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/control_verbs.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/Utf8ComponentClass.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/buildstate.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/buildstate.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/check_refs.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/check_refs.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/logical_combination.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/parse_error.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/parser_util.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/Parser.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/prefilter.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/shortcut_literal.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/ucp_table.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/unsupported.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/utf8_validate.cpp"
|
||||
"${LIBRARY_DIR}/src/parser/Utf8ComponentClass.cpp"
|
||||
|
||||
"${LIBRARY_DIR}/src/rose/block.c"
|
||||
"${LIBRARY_DIR}/src/rose/catchup.c"
|
||||
"${LIBRARY_DIR}/src/rose/init.c"
|
||||
"${LIBRARY_DIR}/src/rose/match.c"
|
||||
"${LIBRARY_DIR}/src/rose/program_runtime.c"
|
||||
"${LIBRARY_DIR}/src/rose/rose_build_add_mask.cpp"
|
||||
"${LIBRARY_DIR}/src/rose/rose_build_add.cpp"
|
||||
"${LIBRARY_DIR}/src/rose/rose_build_add_mask.cpp"
|
||||
"${LIBRARY_DIR}/src/rose/rose_build_anchored.cpp"
|
||||
"${LIBRARY_DIR}/src/rose/rose_build_bytecode.cpp"
|
||||
"${LIBRARY_DIR}/src/rose/rose_build_castle.cpp"
|
||||
@ -187,53 +205,95 @@ set (SRCS
|
||||
"${LIBRARY_DIR}/src/rose/rose_build_matchers.cpp"
|
||||
"${LIBRARY_DIR}/src/rose/rose_build_merge.cpp"
|
||||
"${LIBRARY_DIR}/src/rose/rose_build_misc.cpp"
|
||||
"${LIBRARY_DIR}/src/rose/rose_build_misc.cpp"
|
||||
"${LIBRARY_DIR}/src/rose/rose_build_program.cpp"
|
||||
"${LIBRARY_DIR}/src/rose/rose_build_role_aliasing.cpp"
|
||||
"${LIBRARY_DIR}/src/rose/rose_build_scatter.cpp"
|
||||
"${LIBRARY_DIR}/src/rose/rose_build_width.cpp"
|
||||
"${LIBRARY_DIR}/src/rose/rose_in_util.cpp"
|
||||
"${LIBRARY_DIR}/src/rose/stream.c"
|
||||
"${LIBRARY_DIR}/src/runtime.c"
|
||||
"${LIBRARY_DIR}/src/scratch.c"
|
||||
|
||||
"${LIBRARY_DIR}/src/smallwrite/smallwrite_build.cpp"
|
||||
|
||||
"${LIBRARY_DIR}/src/som/slot_manager.cpp"
|
||||
"${LIBRARY_DIR}/src/som/som_runtime.c"
|
||||
"${LIBRARY_DIR}/src/som/som_stream.c"
|
||||
"${LIBRARY_DIR}/src/stream_compress.c"
|
||||
|
||||
"${LIBRARY_DIR}/src/util/alloc.cpp"
|
||||
"${LIBRARY_DIR}/src/util/charreach.cpp"
|
||||
"${LIBRARY_DIR}/src/util/clique.cpp"
|
||||
"${LIBRARY_DIR}/src/util/compile_context.cpp"
|
||||
"${LIBRARY_DIR}/src/util/compile_error.cpp"
|
||||
"${LIBRARY_DIR}/src/util/cpuid_flags.c"
|
||||
"${LIBRARY_DIR}/src/util/depth.cpp"
|
||||
"${LIBRARY_DIR}/src/util/fatbit_build.cpp"
|
||||
"${LIBRARY_DIR}/src/util/multibit_build.cpp"
|
||||
"${LIBRARY_DIR}/src/util/multibit.c"
|
||||
"${LIBRARY_DIR}/src/util/multibit_build.cpp"
|
||||
"${LIBRARY_DIR}/src/util/report_manager.cpp"
|
||||
"${LIBRARY_DIR}/src/util/simd_utils.c"
|
||||
"${LIBRARY_DIR}/src/util/state_compress.c"
|
||||
"${LIBRARY_DIR}/src/util/target_info.cpp"
|
||||
"${LIBRARY_DIR}/src/util/ue2string.cpp"
|
||||
)
|
||||
|
||||
add_library (_hyperscan ${SRCS})
|
||||
# The original build system invokes ragel on src/parser/{Parser|control_verbs}.rl (+ a few more .rl files which are unneeded). To avoid a
|
||||
# build-time dependency on ragel (via contrib/ or find_program()), add the manually generated output of ragel to the sources.
|
||||
# Please regenerate these files if you update vectorscan.
|
||||
list (APPEND SRCS
|
||||
"${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/Parser.cpp"
|
||||
"${LIBRARY_DIR}/../vectorscan-cmake/rageled_files/control_verbs.cpp"
|
||||
)
|
||||
|
||||
target_compile_options (_hyperscan
|
||||
PRIVATE -g0 # Library has too much debug information
|
||||
-mno-avx -mno-avx2 # The library is using dynamic dispatch and is confused if AVX is enabled globally
|
||||
-march=corei7 -O2 -fno-strict-aliasing -fno-omit-frame-pointer -fvisibility=hidden # The options from original build system
|
||||
-fno-sanitize=undefined # Assume the library takes care of itself
|
||||
)
|
||||
target_include_directories (_hyperscan
|
||||
PRIVATE
|
||||
common
|
||||
"${LIBRARY_DIR}/include"
|
||||
)
|
||||
target_include_directories (_hyperscan SYSTEM PUBLIC "${LIBRARY_DIR}/src")
|
||||
# Platform-dependent files
|
||||
if (ARCH_AMD64)
|
||||
target_include_directories (_hyperscan PRIVATE x86_64)
|
||||
endif ()
|
||||
target_link_libraries (_hyperscan PRIVATE boost::headers_only)
|
||||
list(APPEND SRCS
|
||||
"${LIBRARY_DIR}/src/util/arch/x86/cpuid_flags.c"
|
||||
"${LIBRARY_DIR}/src/util/arch/x86/masked_move.c"
|
||||
"${LIBRARY_DIR}/src/util/supervector/arch/x86/impl.cpp"
|
||||
)
|
||||
endif()
|
||||
|
||||
add_library (ch_contrib::hyperscan ALIAS _hyperscan)
|
||||
if (ARCH_AARCH64)
|
||||
list(APPEND SRCS
|
||||
"${LIBRARY_DIR}/src/util/arch/arm/cpuid_flags.c"
|
||||
"${LIBRARY_DIR}/src/util/supervector/arch/arm/impl.cpp"
|
||||
)
|
||||
endif()
|
||||
|
||||
# TODO
|
||||
# if (ARCH_PPC64LE)
|
||||
# list(APPEND SRCS
|
||||
# "${LIBRARY_DIR}/src/util/supervector/arch/ppc64el/impl.cpp"
|
||||
# )
|
||||
# endif()
|
||||
|
||||
add_library (_vectorscan ${SRCS})
|
||||
|
||||
target_compile_options (_vectorscan PRIVATE
|
||||
-g0 # library has too much debug information
|
||||
-fno-sanitize=undefined # assume the library takes care of itself
|
||||
-O2 -fno-strict-aliasing -fno-omit-frame-pointer -fvisibility=hidden # options from original build system
|
||||
)
|
||||
|
||||
# Include version header manually generated by running the original build system
|
||||
target_include_directories (_vectorscan SYSTEM PRIVATE common)
|
||||
|
||||
# vectorscan inherited some patched in-source versions of boost headers to fix a bug in
|
||||
# boost 1.69. This bug has been solved long ago but vectorscan's source code still
|
||||
# points to the patched versions, so include it here.
|
||||
target_include_directories (_vectorscan SYSTEM PRIVATE "${LIBRARY_DIR}/include")
|
||||
|
||||
target_include_directories (_vectorscan SYSTEM PUBLIC "${LIBRARY_DIR}/src")
|
||||
|
||||
# Include platform-specific config header generated by manually running the original build system
|
||||
# Please regenerate these files if you update vectorscan.
|
||||
|
||||
if (ARCH_AMD64)
|
||||
target_include_directories (_vectorscan PRIVATE x86_64)
|
||||
endif ()
|
||||
|
||||
if (ARCH_AARCH64)
|
||||
target_include_directories (_vectorscan PRIVATE aarch64)
|
||||
endif ()
|
||||
|
||||
target_link_libraries (_vectorscan PRIVATE boost::headers_only)
|
||||
|
||||
add_library (ch_contrib::vectorscan ALIAS _vectorscan)
|
142
contrib/vectorscan-cmake/aarch64/config.h
Normal file
142
contrib/vectorscan-cmake/aarch64/config.h
Normal file
@ -0,0 +1,142 @@
|
||||
/* used by cmake */
|
||||
|
||||
#ifndef CONFIG_H_
|
||||
#define CONFIG_H_
|
||||
|
||||
/* "Define if the build is 32 bit" */
|
||||
/* #undef ARCH_32_BIT */
|
||||
|
||||
/* "Define if the build is 64 bit" */
|
||||
#define ARCH_64_BIT
|
||||
|
||||
/* "Define if building for IA32" */
|
||||
/* #undef ARCH_IA32 */
|
||||
|
||||
/* "Define if building for EM64T" */
|
||||
/* #undef ARCH_X86_64 */
|
||||
|
||||
/* "Define if building for ARM32" */
|
||||
/* #undef ARCH_ARM32 */
|
||||
|
||||
/* "Define if building for AARCH64" */
|
||||
#define ARCH_AARCH64
|
||||
|
||||
/* "Define if building for PPC64EL" */
|
||||
/* #undef ARCH_PPC64EL */
|
||||
|
||||
/* "Define if cross compiling for AARCH64" */
|
||||
/* #undef CROSS_COMPILE_AARCH64 */
|
||||
|
||||
/* Define if building SVE for AARCH64. */
|
||||
/* #undef BUILD_SVE */
|
||||
|
||||
/* Define if building SVE2 for AARCH64. */
|
||||
/* #undef BUILD_SVE2 */
|
||||
|
||||
/* Define if building SVE2+BITPERM for AARCH64. */
|
||||
/* #undef BUILD_SVE2_BITPERM */
|
||||
|
||||
/* internal build, switch on dump support. */
|
||||
/* #undef DUMP_SUPPORT */
|
||||
|
||||
/* Define if building "fat" runtime. */
|
||||
/* #undef FAT_RUNTIME */
|
||||
|
||||
/* Define if building AVX2 in the fat runtime. */
|
||||
/* #undef BUILD_AVX2 */
|
||||
|
||||
/* Define if building AVX-512 in the fat runtime. */
|
||||
/* #undef BUILD_AVX512 */
|
||||
|
||||
/* Define if building AVX512VBMI in the fat runtime. */
|
||||
/* #undef BUILD_AVX512VBMI */
|
||||
|
||||
/* Define to 1 if `backtrace' works. */
|
||||
#define HAVE_BACKTRACE
|
||||
|
||||
/* C compiler has __builtin_assume_aligned */
|
||||
#define HAVE_CC_BUILTIN_ASSUME_ALIGNED
|
||||
|
||||
/* C++ compiler has __builtin_assume_aligned */
|
||||
#define HAVE_CXX_BUILTIN_ASSUME_ALIGNED
|
||||
|
||||
/* C++ compiler has x86intrin.h */
|
||||
/* #undef HAVE_CXX_X86INTRIN_H */
|
||||
|
||||
/* C compiler has x86intrin.h */
|
||||
/* #undef HAVE_C_X86INTRIN_H */
|
||||
|
||||
/* C++ compiler has intrin.h */
|
||||
/* #undef HAVE_CXX_INTRIN_H */
|
||||
|
||||
/* C compiler has intrin.h */
|
||||
/* #undef HAVE_C_INTRIN_H */
|
||||
|
||||
/* C compiler has arm_neon.h */
|
||||
#define HAVE_C_ARM_NEON_H
|
||||
|
||||
/* C compiler has arm_sve.h */
|
||||
/* #undef HAVE_C_ARM_SVE_H */
|
||||
|
||||
/* C compiler has arm_neon.h */
|
||||
/* #undef HAVE_C_PPC64EL_ALTIVEC_H */
|
||||
|
||||
/* Define to 1 if you have the declaration of `pthread_setaffinity_np', and to
|
||||
0 if you don't. */
|
||||
/* #undef HAVE_DECL_PTHREAD_SETAFFINITY_NP */
|
||||
|
||||
/* #undef HAVE_PTHREAD_NP_H */
|
||||
|
||||
/* Define to 1 if you have the `malloc_info' function. */
|
||||
/* #undef HAVE_MALLOC_INFO */
|
||||
|
||||
/* Define to 1 if you have the `memmem' function. */
|
||||
/* #undef HAVE_MEMMEM */
|
||||
|
||||
/* Define to 1 if you have a working `mmap' system call. */
|
||||
#define HAVE_MMAP
|
||||
|
||||
/* Define to 1 if `posix_memalign' works. */
|
||||
#define HAVE_POSIX_MEMALIGN
|
||||
|
||||
/* Define to 1 if you have the `setrlimit' function. */
|
||||
#define HAVE_SETRLIMIT
|
||||
|
||||
/* Define to 1 if you have the `shmget' function. */
|
||||
/* #undef HAVE_SHMGET */
|
||||
|
||||
/* Define to 1 if you have the `sigaction' function. */
|
||||
#define HAVE_SIGACTION
|
||||
|
||||
/* Define to 1 if you have the `sigaltstack' function. */
|
||||
#define HAVE_SIGALTSTACK
|
||||
|
||||
/* Define if the sqlite3_open_v2 call is available */
|
||||
/* #undef HAVE_SQLITE3_OPEN_V2 */
|
||||
|
||||
/* Define to 1 if you have the <unistd.h> header file. */
|
||||
#define HAVE_UNISTD_H
|
||||
|
||||
/* Define to 1 if you have the `_aligned_malloc' function. */
|
||||
/* #undef HAVE__ALIGNED_MALLOC */
|
||||
|
||||
/* Define if compiler has __builtin_constant_p */
|
||||
/* #undef HAVE__BUILTIN_CONSTANT_P */
|
||||
|
||||
/* Optimize, inline critical functions */
|
||||
#define HS_OPTIMIZE
|
||||
|
||||
#define HS_VERSION
|
||||
#define HS_MAJOR_VERSION
|
||||
#define HS_MINOR_VERSION
|
||||
#define HS_PATCH_VERSION
|
||||
|
||||
#define BUILD_DATE
|
||||
|
||||
/* define if this is a release build. */
|
||||
#define RELEASE_BUILD
|
||||
|
||||
/* define if reverse_graph requires patch for boost 1.62.0 */
|
||||
/* #undef BOOST_REVGRAPH_PATCH */
|
||||
|
||||
#endif /* CONFIG_H_ */
|
@ -32,9 +32,8 @@
|
||||
/**
|
||||
* A version string to identify this release of Hyperscan.
|
||||
*/
|
||||
#define HS_VERSION_STRING "5.1.1 2000-01-01"
|
||||
#define HS_VERSION_STRING "5.4.7 2022-06-20"
|
||||
|
||||
#define HS_VERSION_32BIT ((5 << 24) | (1 << 16) | (1 << 8) | 0)
|
||||
#define HS_VERSION_32BIT ((5 << 24) | (1 << 16) | (7 << 8) | 0)
|
||||
|
||||
#endif /* HS_VERSION_H_C6428FAF8E3713 */
|
||||
|
5605
contrib/vectorscan-cmake/rageled_files/Parser.cpp
Normal file
5605
contrib/vectorscan-cmake/rageled_files/Parser.cpp
Normal file
File diff suppressed because it is too large
Load Diff
443
contrib/vectorscan-cmake/rageled_files/control_verbs.cpp
Normal file
443
contrib/vectorscan-cmake/rageled_files/control_verbs.cpp
Normal file
@ -0,0 +1,443 @@
|
||||
|
||||
#line 1 "control_verbs.rl"
|
||||
/*
|
||||
* Copyright (c) 2017, Intel Corporation
|
||||
*
|
||||
* Redistribution and use in source and binary forms, with or without
|
||||
* modification, are permitted provided that the following conditions are met:
|
||||
*
|
||||
* * Redistributions of source code must retain the above copyright notice,
|
||||
* this list of conditions and the following disclaimer.
|
||||
* * Redistributions in binary form must reproduce the above copyright
|
||||
* notice, this list of conditions and the following disclaimer in the
|
||||
* documentation and/or other materials provided with the distribution.
|
||||
* * Neither the name of Intel Corporation nor the names of its contributors
|
||||
* may be used to endorse or promote products derived from this software
|
||||
* without specific prior written permission.
|
||||
*
|
||||
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
|
||||
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
|
||||
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
|
||||
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
|
||||
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
|
||||
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
|
||||
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
|
||||
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
|
||||
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
|
||||
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
|
||||
* POSSIBILITY OF SUCH DAMAGE.
|
||||
*/
|
||||
|
||||
/**
|
||||
* \file
|
||||
* \brief Parser for control verbs that can occur at the beginning of a pattern.
|
||||
*/
|
||||
|
||||
#include "parser/control_verbs.h"
|
||||
|
||||
#include "parser/Parser.h"
|
||||
#include "parser/parse_error.h"
|
||||
|
||||
#include <cstring>
|
||||
#include <sstream>
|
||||
|
||||
using namespace std;
|
||||
|
||||
namespace ue2 {
|
||||
|
||||
const char *read_control_verbs(const char *ptr, const char *end, size_t start,
|
||||
ParseMode &mode) {
|
||||
const char *p = ptr;
|
||||
const char *pe = end;
|
||||
const char *eof = pe;
|
||||
const char *ts, *te;
|
||||
int cs;
|
||||
UNUSED int act;
|
||||
|
||||
|
||||
#line 59 "control_verbs.cpp"
|
||||
static const char _ControlVerbs_actions[] = {
|
||||
0, 1, 0, 1, 1, 1, 2, 1,
|
||||
3, 1, 4, 1, 5, 1, 6, 1,
|
||||
7, 1, 8, 1, 9
|
||||
};
|
||||
|
||||
static const unsigned char _ControlVerbs_key_offsets[] = {
|
||||
0, 7, 8, 10, 12, 14, 16, 18,
|
||||
20, 21, 23, 25, 27, 30, 32, 34,
|
||||
36, 38, 40, 42, 44, 46, 48, 50,
|
||||
52, 55, 57, 59, 61, 63, 66, 68,
|
||||
70, 72, 74, 76, 79, 82, 84, 86,
|
||||
88, 90, 92, 94, 96, 98, 100, 102,
|
||||
105, 107, 109, 111, 113, 115, 117, 119,
|
||||
121, 123, 125, 127, 129, 131, 133, 135,
|
||||
137, 139, 141, 143, 146, 148, 149, 151,
|
||||
155, 157, 159, 160, 161
|
||||
};
|
||||
|
||||
static const char _ControlVerbs_trans_keys[] = {
|
||||
41, 65, 66, 67, 76, 78, 85, 41,
|
||||
41, 78, 41, 89, 41, 67, 41, 82,
|
||||
41, 76, 41, 70, 41, 41, 83, 41,
|
||||
82, 41, 95, 41, 65, 85, 41, 78,
|
||||
41, 89, 41, 67, 41, 78, 41, 73,
|
||||
41, 67, 41, 79, 41, 68, 41, 69,
|
||||
41, 82, 41, 76, 41, 70, 73, 41,
|
||||
77, 41, 73, 41, 84, 41, 95, 41,
|
||||
77, 82, 41, 65, 41, 84, 41, 67,
|
||||
41, 72, 41, 61, 41, 48, 57, 41,
|
||||
48, 57, 41, 69, 41, 67, 41, 85,
|
||||
41, 82, 41, 83, 41, 73, 41, 79,
|
||||
41, 78, 41, 79, 41, 95, 41, 65,
|
||||
83, 41, 85, 41, 84, 41, 79, 41,
|
||||
95, 41, 80, 41, 79, 41, 83, 41,
|
||||
83, 41, 69, 41, 83, 41, 83, 41,
|
||||
84, 41, 65, 41, 82, 41, 84, 41,
|
||||
95, 41, 79, 41, 80, 41, 84, 41,
|
||||
67, 84, 41, 80, 41, 41, 70, 41,
|
||||
49, 51, 56, 41, 54, 41, 50, 41,
|
||||
40, 42, 0
|
||||
};
|
||||
|
||||
static const char _ControlVerbs_single_lengths[] = {
|
||||
7, 1, 2, 2, 2, 2, 2, 2,
|
||||
1, 2, 2, 2, 3, 2, 2, 2,
|
||||
2, 2, 2, 2, 2, 2, 2, 2,
|
||||
3, 2, 2, 2, 2, 3, 2, 2,
|
||||
2, 2, 2, 1, 1, 2, 2, 2,
|
||||
2, 2, 2, 2, 2, 2, 2, 3,
|
||||
2, 2, 2, 2, 2, 2, 2, 2,
|
||||
2, 2, 2, 2, 2, 2, 2, 2,
|
||||
2, 2, 2, 3, 2, 1, 2, 4,
|
||||
2, 2, 1, 1, 1
|
||||
};
|
||||
|
||||
static const char _ControlVerbs_range_lengths[] = {
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 1, 1, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0
|
||||
};
|
||||
|
||||
static const short _ControlVerbs_index_offsets[] = {
|
||||
0, 8, 10, 13, 16, 19, 22, 25,
|
||||
28, 30, 33, 36, 39, 43, 46, 49,
|
||||
52, 55, 58, 61, 64, 67, 70, 73,
|
||||
76, 80, 83, 86, 89, 92, 96, 99,
|
||||
102, 105, 108, 111, 114, 117, 120, 123,
|
||||
126, 129, 132, 135, 138, 141, 144, 147,
|
||||
151, 154, 157, 160, 163, 166, 169, 172,
|
||||
175, 178, 181, 184, 187, 190, 193, 196,
|
||||
199, 202, 205, 208, 212, 215, 217, 220,
|
||||
225, 228, 231, 233, 235
|
||||
};
|
||||
|
||||
static const char _ControlVerbs_indicies[] = {
|
||||
0, 2, 3, 4, 5, 6, 7, 1,
|
||||
8, 1, 8, 9, 1, 8, 10, 1,
|
||||
11, 12, 1, 8, 13, 1, 8, 14,
|
||||
1, 8, 15, 1, 11, 1, 8, 16,
|
||||
1, 8, 17, 1, 8, 18, 1, 8,
|
||||
19, 20, 1, 8, 21, 1, 8, 22,
|
||||
1, 8, 12, 1, 8, 23, 1, 8,
|
||||
24, 1, 8, 25, 1, 8, 26, 1,
|
||||
8, 27, 1, 8, 15, 1, 8, 28,
|
||||
1, 11, 14, 1, 8, 15, 29, 1,
|
||||
8, 30, 1, 8, 31, 1, 8, 32,
|
||||
1, 8, 33, 1, 8, 34, 35, 1,
|
||||
8, 36, 1, 8, 37, 1, 8, 38,
|
||||
1, 8, 39, 1, 8, 40, 1, 8,
|
||||
41, 1, 11, 41, 1, 8, 42, 1,
|
||||
8, 43, 1, 8, 44, 1, 8, 45,
|
||||
1, 8, 46, 1, 8, 47, 1, 8,
|
||||
48, 1, 8, 39, 1, 8, 49, 1,
|
||||
8, 50, 1, 8, 51, 52, 1, 8,
|
||||
53, 1, 8, 54, 1, 8, 55, 1,
|
||||
8, 56, 1, 8, 57, 1, 8, 58,
|
||||
1, 8, 59, 1, 8, 60, 1, 8,
|
||||
61, 1, 8, 62, 1, 8, 15, 1,
|
||||
8, 63, 1, 8, 64, 1, 8, 65,
|
||||
1, 8, 66, 1, 8, 67, 1, 8,
|
||||
68, 1, 8, 69, 1, 8, 15, 1,
|
||||
8, 70, 71, 1, 8, 72, 1, 73,
|
||||
1, 8, 74, 1, 75, 76, 77, 78,
|
||||
1, 8, 15, 1, 8, 15, 1, 75,
|
||||
1, 80, 79, 82, 81, 0
|
||||
};
|
||||
|
||||
static const char _ControlVerbs_trans_targs[] = {
|
||||
75, 1, 2, 9, 22, 24, 45, 67,
|
||||
75, 3, 4, 75, 5, 6, 7, 8,
|
||||
10, 11, 12, 13, 16, 14, 15, 17,
|
||||
18, 19, 20, 21, 23, 25, 26, 27,
|
||||
28, 29, 30, 37, 31, 32, 33, 34,
|
||||
35, 36, 38, 39, 40, 41, 42, 43,
|
||||
44, 46, 47, 48, 59, 49, 50, 51,
|
||||
52, 53, 54, 55, 56, 57, 58, 60,
|
||||
61, 62, 63, 64, 65, 66, 68, 70,
|
||||
69, 75, 71, 75, 72, 73, 74, 75,
|
||||
76, 75, 0
|
||||
};
|
||||
|
||||
static const char _ControlVerbs_trans_actions[] = {
|
||||
19, 0, 0, 0, 0, 0, 0, 0,
|
||||
13, 0, 0, 11, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 9, 0, 7, 0, 0, 0, 15,
|
||||
5, 17, 0
|
||||
};
|
||||
|
||||
static const char _ControlVerbs_to_state_actions[] = {
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 1, 0
|
||||
};
|
||||
|
||||
static const char _ControlVerbs_from_state_actions[] = {
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 0, 0, 0, 0, 0,
|
||||
0, 0, 0, 3, 0
|
||||
};
|
||||
|
||||
static const short _ControlVerbs_eof_trans[] = {
|
||||
1, 1, 1, 1, 1, 1, 1, 1,
|
||||
1, 1, 1, 1, 1, 1, 1, 1,
|
||||
1, 1, 1, 1, 1, 1, 1, 1,
|
||||
1, 1, 1, 1, 1, 1, 1, 1,
|
||||
1, 1, 1, 1, 1, 1, 1, 1,
|
||||
1, 1, 1, 1, 1, 1, 1, 1,
|
||||
1, 1, 1, 1, 1, 1, 1, 1,
|
||||
1, 1, 1, 1, 1, 1, 1, 1,
|
||||
1, 1, 1, 1, 1, 1, 1, 1,
|
||||
1, 1, 1, 0, 82
|
||||
};
|
||||
|
||||
static const int ControlVerbs_start = 75;
|
||||
static const int ControlVerbs_first_final = 75;
|
||||
static const int ControlVerbs_error = -1;
|
||||
|
||||
static const int ControlVerbs_en_main = 75;
|
||||
|
||||
|
||||
#line 249 "control_verbs.cpp"
|
||||
{
|
||||
cs = ControlVerbs_start;
|
||||
ts = 0;
|
||||
te = 0;
|
||||
act = 0;
|
||||
}
|
||||
|
||||
#line 105 "control_verbs.rl"
|
||||
|
||||
|
||||
try {
|
||||
|
||||
#line 262 "control_verbs.cpp"
|
||||
{
|
||||
int _klen;
|
||||
unsigned int _trans;
|
||||
const char *_acts;
|
||||
unsigned int _nacts;
|
||||
const char *_keys;
|
||||
|
||||
if ( p == pe )
|
||||
goto _test_eof;
|
||||
_resume:
|
||||
_acts = _ControlVerbs_actions + _ControlVerbs_from_state_actions[cs];
|
||||
_nacts = (unsigned int) *_acts++;
|
||||
while ( _nacts-- > 0 ) {
|
||||
switch ( *_acts++ ) {
|
||||
case 1:
|
||||
#line 1 "NONE"
|
||||
{ts = p;}
|
||||
break;
|
||||
#line 281 "control_verbs.cpp"
|
||||
}
|
||||
}
|
||||
|
||||
_keys = _ControlVerbs_trans_keys + _ControlVerbs_key_offsets[cs];
|
||||
_trans = _ControlVerbs_index_offsets[cs];
|
||||
|
||||
_klen = _ControlVerbs_single_lengths[cs];
|
||||
if ( _klen > 0 ) {
|
||||
const char *_lower = _keys;
|
||||
const char *_mid;
|
||||
const char *_upper = _keys + _klen - 1;
|
||||
while (1) {
|
||||
if ( _upper < _lower )
|
||||
break;
|
||||
|
||||
_mid = _lower + ((_upper-_lower) >> 1);
|
||||
if ( (*p) < *_mid )
|
||||
_upper = _mid - 1;
|
||||
else if ( (*p) > *_mid )
|
||||
_lower = _mid + 1;
|
||||
else {
|
||||
_trans += (unsigned int)(_mid - _keys);
|
||||
goto _match;
|
||||
}
|
||||
}
|
||||
_keys += _klen;
|
||||
_trans += _klen;
|
||||
}
|
||||
|
||||
_klen = _ControlVerbs_range_lengths[cs];
|
||||
if ( _klen > 0 ) {
|
||||
const char *_lower = _keys;
|
||||
const char *_mid;
|
||||
const char *_upper = _keys + (_klen<<1) - 2;
|
||||
while (1) {
|
||||
if ( _upper < _lower )
|
||||
break;
|
||||
|
||||
_mid = _lower + (((_upper-_lower) >> 1) & ~1);
|
||||
if ( (*p) < _mid[0] )
|
||||
_upper = _mid - 2;
|
||||
else if ( (*p) > _mid[1] )
|
||||
_lower = _mid + 2;
|
||||
else {
|
||||
_trans += (unsigned int)((_mid - _keys)>>1);
|
||||
goto _match;
|
||||
}
|
||||
}
|
||||
_trans += _klen;
|
||||
}
|
||||
|
||||
_match:
|
||||
_trans = _ControlVerbs_indicies[_trans];
|
||||
_eof_trans:
|
||||
cs = _ControlVerbs_trans_targs[_trans];
|
||||
|
||||
if ( _ControlVerbs_trans_actions[_trans] == 0 )
|
||||
goto _again;
|
||||
|
||||
_acts = _ControlVerbs_actions + _ControlVerbs_trans_actions[_trans];
|
||||
_nacts = (unsigned int) *_acts++;
|
||||
while ( _nacts-- > 0 )
|
||||
{
|
||||
switch ( *_acts++ )
|
||||
{
|
||||
case 2:
|
||||
#line 1 "NONE"
|
||||
{te = p+1;}
|
||||
break;
|
||||
case 3:
|
||||
#line 76 "control_verbs.rl"
|
||||
{te = p+1;{
|
||||
mode.utf8 = true;
|
||||
}}
|
||||
break;
|
||||
case 4:
|
||||
#line 80 "control_verbs.rl"
|
||||
{te = p+1;{
|
||||
mode.ucp = true;
|
||||
}}
|
||||
break;
|
||||
case 5:
|
||||
#line 84 "control_verbs.rl"
|
||||
{te = p+1;{
|
||||
ostringstream str;
|
||||
str << "Unsupported control verb " << string(ts, te - ts);
|
||||
throw LocatedParseError(str.str());
|
||||
}}
|
||||
break;
|
||||
case 6:
|
||||
#line 90 "control_verbs.rl"
|
||||
{te = p+1;{
|
||||
ostringstream str;
|
||||
str << "Unknown control verb " << string(ts, te - ts);
|
||||
throw LocatedParseError(str.str());
|
||||
}}
|
||||
break;
|
||||
case 7:
|
||||
#line 97 "control_verbs.rl"
|
||||
{te = p+1;{
|
||||
p--;
|
||||
{p++; goto _out; }
|
||||
}}
|
||||
break;
|
||||
case 8:
|
||||
#line 97 "control_verbs.rl"
|
||||
{te = p;p--;{
|
||||
p--;
|
||||
{p++; goto _out; }
|
||||
}}
|
||||
break;
|
||||
case 9:
|
||||
#line 97 "control_verbs.rl"
|
||||
{{p = ((te))-1;}{
|
||||
p--;
|
||||
{p++; goto _out; }
|
||||
}}
|
||||
break;
|
||||
#line 400 "control_verbs.cpp"
|
||||
}
|
||||
}
|
||||
|
||||
_again:
|
||||
_acts = _ControlVerbs_actions + _ControlVerbs_to_state_actions[cs];
|
||||
_nacts = (unsigned int) *_acts++;
|
||||
while ( _nacts-- > 0 ) {
|
||||
switch ( *_acts++ ) {
|
||||
case 0:
|
||||
#line 1 "NONE"
|
||||
{ts = 0;}
|
||||
break;
|
||||
#line 413 "control_verbs.cpp"
|
||||
}
|
||||
}
|
||||
|
||||
if ( ++p != pe )
|
||||
goto _resume;
|
||||
_test_eof: {}
|
||||
if ( p == eof )
|
||||
{
|
||||
if ( _ControlVerbs_eof_trans[cs] > 0 ) {
|
||||
_trans = _ControlVerbs_eof_trans[cs] - 1;
|
||||
goto _eof_trans;
|
||||
}
|
||||
}
|
||||
|
||||
_out: {}
|
||||
}
|
||||
|
||||
#line 109 "control_verbs.rl"
|
||||
} catch (LocatedParseError &error) {
|
||||
if (ts >= ptr && ts <= pe) {
|
||||
error.locate(ts - ptr + start);
|
||||
} else {
|
||||
error.locate(0);
|
||||
}
|
||||
throw;
|
||||
}
|
||||
|
||||
return p;
|
||||
}
|
||||
|
||||
} // namespace ue2
|
@ -15,15 +15,42 @@
|
||||
/* "Define if building for EM64T" */
|
||||
#define ARCH_X86_64
|
||||
|
||||
/* "Define if building for ARM32" */
|
||||
/* #undef ARCH_ARM32 */
|
||||
|
||||
/* "Define if building for AARCH64" */
|
||||
/* #undef ARCH_AARCH64 */
|
||||
|
||||
/* "Define if building for PPC64EL" */
|
||||
/* #undef ARCH_PPC64EL */
|
||||
|
||||
/* "Define if cross compiling for AARCH64" */
|
||||
/* #undef CROSS_COMPILE_AARCH64 */
|
||||
|
||||
/* Define if building SVE for AARCH64. */
|
||||
/* #undef BUILD_SVE */
|
||||
|
||||
/* Define if building SVE2 for AARCH64. */
|
||||
/* #undef BUILD_SVE2 */
|
||||
|
||||
/* Define if building SVE2+BITPERM for AARCH64. */
|
||||
/* #undef BUILD_SVE2_BITPERM */
|
||||
|
||||
/* internal build, switch on dump support. */
|
||||
/* #undef DUMP_SUPPORT */
|
||||
|
||||
/* Define if building "fat" runtime. */
|
||||
/* #undef FAT_RUNTIME */
|
||||
|
||||
/* Define if building AVX2 in the fat runtime. */
|
||||
/* #undef BUILD_AVX2 */
|
||||
|
||||
/* Define if building AVX-512 in the fat runtime. */
|
||||
/* #undef BUILD_AVX512 */
|
||||
|
||||
/* Define if building AVX512VBMI in the fat runtime. */
|
||||
/* #undef BUILD_AVX512VBMI */
|
||||
|
||||
/* Define to 1 if `backtrace' works. */
|
||||
#define HAVE_BACKTRACE
|
||||
|
||||
@ -45,6 +72,15 @@
|
||||
/* C compiler has intrin.h */
|
||||
/* #undef HAVE_C_INTRIN_H */
|
||||
|
||||
/* C compiler has arm_neon.h */
|
||||
/* #undef HAVE_C_ARM_NEON_H */
|
||||
|
||||
/* C compiler has arm_sve.h */
|
||||
/* #undef HAVE_C_ARM_SVE_H */
|
||||
|
||||
/* C compiler has arm_neon.h */
|
||||
/* #undef HAVE_C_PPC64EL_ALTIVEC_H */
|
||||
|
||||
/* Define to 1 if you have the declaration of `pthread_setaffinity_np', and to
|
||||
0 if you don't. */
|
||||
/* #undef HAVE_DECL_PTHREAD_SETAFFINITY_NP */
|
||||
@ -85,7 +121,7 @@
|
||||
/* #undef HAVE__ALIGNED_MALLOC */
|
||||
|
||||
/* Define if compiler has __builtin_constant_p */
|
||||
#define HAVE__BUILTIN_CONSTANT_P
|
||||
/* #undef HAVE__BUILTIN_CONSTANT_P */
|
||||
|
||||
/* Optimize, inline critical functions */
|
||||
#define HS_OPTIMIZE
|
@ -67,7 +67,7 @@ RUN python3 -m pip install \
|
||||
dict2xml \
|
||||
dicttoxml \
|
||||
docker \
|
||||
docker-compose==1.28.2 \
|
||||
docker-compose==1.29.2 \
|
||||
grpcio \
|
||||
grpcio-tools \
|
||||
kafka-python \
|
||||
|
@ -38,7 +38,7 @@ RUN apt-get update \
|
||||
ENV TZ=Europe/Moscow
|
||||
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
|
||||
|
||||
RUN pip3 install urllib3 testflows==1.7.20 docker-compose==1.29.1 docker==5.0.0 dicttoxml kazoo tzlocal==2.1 pytz python-dateutil numpy
|
||||
RUN pip3 install urllib3 testflows==1.7.20 docker-compose==1.29.2 docker==5.0.0 dicttoxml kazoo tzlocal==2.1 pytz python-dateutil numpy
|
||||
|
||||
ENV DOCKER_CHANNEL stable
|
||||
ENV DOCKER_VERSION 20.10.6
|
||||
|
@ -18,6 +18,7 @@ The list of third-party libraries:
|
||||
| aws-c-common | [Apache](https://github.com/ClickHouse-Extras/aws-c-common/blob/736a82d1697c108b04a277e66438a7f4e19b6857/LICENSE) |
|
||||
| aws-c-event-stream | [Apache](https://github.com/ClickHouse-Extras/aws-c-event-stream/blob/3bc33662f9ccff4f4cbcf9509cc78c26e022fde0/LICENSE) |
|
||||
| aws-checksums | [Apache](https://github.com/ClickHouse-Extras/aws-checksums/blob/519d6d9093819b6cf89ffff589a27ef8f83d0f65/LICENSE) |
|
||||
| base58 | [MIT](https://github.com/ClickHouse/base-x/blob/3e58874643c087f57e82b0ff03825c933fab945a/LICENSE) |
|
||||
| base64 | [BSD 2-clause](https://github.com/ClickHouse-Extras/Turbo-Base64/blob/af9b331f2b4f30b41c70f3a571ff904a8251c1d3/LICENSE) |
|
||||
| boost | [Boost](https://github.com/ClickHouse-Extras/boost/blob/9cf09dbfd55a5c6202dedbdf40781a51b02c2675/LICENSE_1_0.txt) |
|
||||
| boringssl | [BSD](https://github.com/ClickHouse-Extras/boringssl/blob/a6a2e2ab3e44d97ce98e51c558e989f211de7eb3/LICENSE) |
|
||||
@ -40,7 +41,7 @@ The list of third-party libraries:
|
||||
| googletest | [BSD 3-clause](https://github.com/google/googletest/blob/e7e591764baba0a0c3c9ad0014430e7a27331d16/LICENSE) |
|
||||
| grpc | [Apache](https://github.com/ClickHouse-Extras/grpc/blob/60c986e15cae70aade721d26badabab1f822fdd6/LICENSE) |
|
||||
| h3 | [Apache](https://github.com/ClickHouse-Extras/h3/blob/c7f46cfd71fb60e2fefc90e28abe81657deff735/LICENSE) |
|
||||
| hyperscan | [Boost](https://github.com/ClickHouse-Extras/hyperscan/blob/e9f08df0213fc637aac0a5bbde9beeaeba2fe9fa/LICENSE) |
|
||||
| vectorscan | [Boost](https://github.com/ClickHouse-Extras/hyperscan/blob/73695e419c27af7fe2a099c7aa57931cc02aea5d/LICENSE) |
|
||||
| icu | [Public Domain](https://github.com/unicode-org/icu/blob/a56dde820dc35665a66f2e9ee8ba58e75049b668/icu4c/LICENSE) |
|
||||
| icudata | [Public Domain](https://github.com/ClickHouse-Extras/icudata/blob/72d9a4a7febc904e2b0a534ccb25ae40fac5f1e5/LICENSE) |
|
||||
| jemalloc | [BSD 2-clause](https://github.com/ClickHouse-Extras/jemalloc/blob/e6891d9746143bf2cf617493d880ba5a0b9a3efd/COPYING) |
|
||||
|
@ -20,6 +20,8 @@ Here is a complete list of available database engines. Follow the links for more
|
||||
|
||||
- [PostgreSQL](../../engines/database-engines/postgresql.md)
|
||||
|
||||
- [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md)
|
||||
|
||||
- [Replicated](../../engines/database-engines/replicated.md)
|
||||
|
||||
- [SQLite](../../engines/database-engines/sqlite.md)
|
||||
|
@ -26,14 +26,30 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo
|
||||
- `user` — MySQL user.
|
||||
- `password` — User password.
|
||||
|
||||
**Engine Settings**
|
||||
## Engine Settings
|
||||
|
||||
- `max_rows_in_buffer` — Maximum number of rows that data is allowed to cache in memory (for single table and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `65 505`.
|
||||
- `max_bytes_in_buffer` — Maximum number of bytes that data is allowed to cache in memory (for single table and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `1 048 576`.
|
||||
- `max_flush_data_time` — Maximum number of milliseconds that data is allowed to cache in memory (for database and the cache data unable to query). When this time is exceeded, the data will be materialized. Default: `1000`.
|
||||
- `max_wait_time_when_mysql_unavailable` — Retry interval when MySQL is not available (milliseconds). Negative value disables retry. Default: `1000`.
|
||||
- `allows_query_when_mysql_lost` — Allows to query a materialized table when MySQL is lost. Default: `0` (`false`).
|
||||
- `materialized_mysql_tables_list` — a comma-separated list of mysql database tables, which will be replicated by MaterializedMySQL database engine. Default value: empty list — means whole tables will be replicated.
|
||||
### max_rows_in_buffer
|
||||
|
||||
`max_rows_in_buffer` — Maximum number of rows that data is allowed to cache in memory (for single table and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `65 505`.
|
||||
|
||||
### max_bytes_in_buffer
|
||||
|
||||
`max_bytes_in_buffer` — Maximum number of bytes that data is allowed to cache in memory (for single table and the cache data unable to query). When this number is exceeded, the data will be materialized. Default: `1 048 576`.
|
||||
|
||||
### max_flush_data_time
|
||||
|
||||
`max_flush_data_time` — Maximum number of milliseconds that data is allowed to cache in memory (for database and the cache data unable to query). When this time is exceeded, the data will be materialized. Default: `1000`.
|
||||
|
||||
### max_wait_time_when_mysql_unavailable
|
||||
|
||||
`max_wait_time_when_mysql_unavailable` — Retry interval when MySQL is not available (milliseconds). Negative value disables retry. Default: `1000`.
|
||||
|
||||
### allows_query_when_mysql_lost
|
||||
`allows_query_when_mysql_lost` — Allows to query a materialized table when MySQL is lost. Default: `0` (`false`).
|
||||
|
||||
### materialized_mysql_tables_list
|
||||
|
||||
`materialized_mysql_tables_list` — a comma-separated list of mysql database tables, which will be replicated by MaterializedMySQL database engine. Default value: empty list — means whole tables will be replicated.
|
||||
|
||||
```sql
|
||||
CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***')
|
||||
@ -42,12 +58,17 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user',
|
||||
max_wait_time_when_mysql_unavailable=10000;
|
||||
```
|
||||
|
||||
**Settings on MySQL-server Side**
|
||||
## Settings on MySQL-server Side
|
||||
|
||||
For the correct work of `MaterializedMySQL`, there are few mandatory `MySQL`-side configuration settings that must be set:
|
||||
|
||||
- `default_authentication_plugin = mysql_native_password` since `MaterializedMySQL` can only authorize with this method.
|
||||
- `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializedMySQL` replication.
|
||||
### default_authentication_plugin
|
||||
|
||||
`default_authentication_plugin = mysql_native_password` since `MaterializedMySQL` can only authorize with this method.
|
||||
|
||||
### gtid_mode
|
||||
|
||||
`gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializedMySQL` replication.
|
||||
|
||||
:::note
|
||||
While turning on `gtid_mode` you should also specify `enforce_gtid_consistency = on`.
|
||||
@ -57,8 +78,13 @@ While turning on `gtid_mode` you should also specify `enforce_gtid_consistency =
|
||||
|
||||
When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md) tables are used with virtual `_sign` and `_version` columns.
|
||||
|
||||
- `_version` — Transaction counter. Type [UInt64](../../sql-reference/data-types/int-uint.md).
|
||||
- `_sign` — Deletion mark. Type [Int8](../../sql-reference/data-types/int-uint.md). Possible values:
|
||||
### \_version
|
||||
|
||||
`_version` — Transaction counter. Type [UInt64](../../sql-reference/data-types/int-uint.md).
|
||||
|
||||
### \_sign
|
||||
|
||||
`_sign` — Deletion mark. Type [Int8](../../sql-reference/data-types/int-uint.md). Possible values:
|
||||
- `1` — Row is not deleted,
|
||||
- `-1` — Row is deleted.
|
||||
|
||||
|
@ -150,21 +150,21 @@ Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.htm
|
||||
|
||||
## Settings {#settings}
|
||||
|
||||
1. `materialized_postgresql_tables_list` {#materialized-postgresql-tables-list}
|
||||
### `materialized_postgresql_tables_list` {#materialized-postgresql-tables-list}
|
||||
|
||||
Sets a comma-separated list of PostgreSQL database tables, which will be replicated via [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) database engine.
|
||||
|
||||
Default value: empty list — means whole PostgreSQL database will be replicated.
|
||||
|
||||
2. `materialized_postgresql_schema` {#materialized-postgresql-schema}
|
||||
### `materialized_postgresql_schema` {#materialized-postgresql-schema}
|
||||
|
||||
Default value: empty string. (Default schema is used)
|
||||
|
||||
3. `materialized_postgresql_schema_list` {#materialized-postgresql-schema-list}
|
||||
### `materialized_postgresql_schema_list` {#materialized-postgresql-schema-list}
|
||||
|
||||
Default value: empty list. (Default schema is used)
|
||||
|
||||
4. `materialized_postgresql_allow_automatic_update` {#materialized-postgresql-allow-automatic-update}
|
||||
### `materialized_postgresql_allow_automatic_update` {#materialized-postgresql-allow-automatic-update}
|
||||
|
||||
Do not use this setting before 22.1 version.
|
||||
|
||||
@ -177,7 +177,7 @@ Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.htm
|
||||
|
||||
Default value: `0`.
|
||||
|
||||
5. `materialized_postgresql_max_block_size` {#materialized-postgresql-max-block-size}
|
||||
### `materialized_postgresql_max_block_size` {#materialized-postgresql-max-block-size}
|
||||
|
||||
Sets the number of rows collected in memory before flushing data into PostgreSQL database table.
|
||||
|
||||
@ -187,11 +187,11 @@ Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.htm
|
||||
|
||||
Default value: `65536`.
|
||||
|
||||
6. `materialized_postgresql_replication_slot` {#materialized-postgresql-replication-slot}
|
||||
### `materialized_postgresql_replication_slot` {#materialized-postgresql-replication-slot}
|
||||
|
||||
A user-created replication slot. Must be used together with `materialized_postgresql_snapshot`.
|
||||
|
||||
7. `materialized_postgresql_snapshot` {#materialized-postgresql-snapshot}
|
||||
### `materialized_postgresql_snapshot` {#materialized-postgresql-snapshot}
|
||||
|
||||
A text string identifying a snapshot, from which [initial dump of PostgreSQL tables](../../engines/database-engines/materialized-postgresql.md) will be performed. Must be used together with `materialized_postgresql_replication_slot`.
|
||||
|
||||
|
@ -11,8 +11,8 @@ You can use `AggregatingMergeTree` tables for incremental data aggregation, incl
|
||||
|
||||
The engine processes all columns with the following types:
|
||||
|
||||
- [AggregateFunction](../../../sql-reference/data-types/aggregatefunction.md)
|
||||
- [SimpleAggregateFunction](../../../sql-reference/data-types/simpleaggregatefunction.md)
|
||||
## [AggregateFunction](../../../sql-reference/data-types/aggregatefunction.md)
|
||||
## [SimpleAggregateFunction](../../../sql-reference/data-types/simpleaggregatefunction.md)
|
||||
|
||||
It is appropriate to use `AggregatingMergeTree` if it reduces the number of rows by orders.
|
||||
|
||||
@ -36,7 +36,7 @@ For a description of request parameters, see [request description](../../../sql-
|
||||
|
||||
**Query clauses**
|
||||
|
||||
When creating a `AggregatingMergeTree` table the same [clauses](../../../engines/table-engines/mergetree-family/mergetree.md) are required, as when creating a `MergeTree` table.
|
||||
When creating an `AggregatingMergeTree` table the same [clauses](../../../engines/table-engines/mergetree-family/mergetree.md) are required, as when creating a `MergeTree` table.
|
||||
|
||||
<details markdown="1">
|
||||
|
||||
|
@ -7,7 +7,7 @@ sidebar_label: CollapsingMergeTree
|
||||
|
||||
The engine inherits from [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md) and adds the logic of rows collapsing to data parts merge algorithm.
|
||||
|
||||
`CollapsingMergeTree` asynchronously deletes (collapses) pairs of rows if all of the fields in a sorting key (`ORDER BY`) are equivalent excepting the particular field `Sign` which can have `1` and `-1` values. Rows without a pair are kept. For more details see the [Collapsing](#table_engine-collapsingmergetree-collapsing) section of the document.
|
||||
`CollapsingMergeTree` asynchronously deletes (collapses) pairs of rows if all of the fields in a sorting key (`ORDER BY`) are equivalent except the particular field `Sign`, which can have `1` and `-1` values. Rows without a pair are kept. For more details see the [Collapsing](#table_engine-collapsingmergetree-collapsing) section of the document.
|
||||
|
||||
The engine may significantly reduce the volume of storage and increase the efficiency of `SELECT` query as a consequence.
|
||||
|
||||
@ -28,13 +28,15 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
|
||||
For a description of query parameters, see [query description](../../../sql-reference/statements/create/table.md).
|
||||
|
||||
**CollapsingMergeTree Parameters**
|
||||
## CollapsingMergeTree Parameters
|
||||
|
||||
- `sign` — Name of the column with the type of row: `1` is a “state” row, `-1` is a “cancel” row.
|
||||
### sign
|
||||
|
||||
`sign` — Name of the column with the type of row: `1` is a “state” row, `-1` is a “cancel” row.
|
||||
|
||||
Column data type — `Int8`.
|
||||
|
||||
**Query clauses**
|
||||
## Query clauses
|
||||
|
||||
When creating a `CollapsingMergeTree` table, the same [query clauses](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) are required, as when creating a `MergeTree` table.
|
||||
|
||||
|
@ -87,10 +87,18 @@ Rollup configuration structure:
|
||||
|
||||
### Required Columns {#required-columns}
|
||||
|
||||
- `path_column_name` — The name of the column storing the metric name (Graphite sensor). Default value: `Path`.
|
||||
- `time_column_name` — The name of the column storing the time of measuring the metric. Default value: `Time`.
|
||||
- `value_column_name` — The name of the column storing the value of the metric at the time set in `time_column_name`. Default value: `Value`.
|
||||
- `version_column_name` — The name of the column storing the version of the metric. Default value: `Timestamp`.
|
||||
#### path_column_name
|
||||
|
||||
`path_column_name` — The name of the column storing the metric name (Graphite sensor). Default value: `Path`.
|
||||
|
||||
#### time_column_name
|
||||
`time_column_name` — The name of the column storing the time of measuring the metric. Default value: `Time`.
|
||||
|
||||
#### value_column_name
|
||||
`value_column_name` — The name of the column storing the value of the metric at the time set in `time_column_name`. Default value: `Value`.
|
||||
|
||||
#### version_column_name
|
||||
`version_column_name` — The name of the column storing the version of the metric. Default value: `Timestamp`.
|
||||
|
||||
### Patterns {#patterns}
|
||||
|
||||
@ -254,7 +262,6 @@ Valid values:
|
||||
</graphite_rollup>
|
||||
```
|
||||
|
||||
|
||||
:::warning
|
||||
Data rollup is performed during merges. Usually, for old partitions, merges are not started, so for rollup it is necessary to trigger an unscheduled merge using [optimize](../../../sql-reference/statements/optimize.md). Or use additional tools, for example [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer).
|
||||
:::
|
||||
|
@ -60,9 +60,13 @@ For a description of parameters, see the [CREATE query description](../../../sql
|
||||
|
||||
### Query Clauses {#mergetree-query-clauses}
|
||||
|
||||
- `ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine does not have parameters.
|
||||
#### ENGINE
|
||||
|
||||
- `ORDER BY` — The sorting key.
|
||||
`ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine does not have parameters.
|
||||
|
||||
#### ORDER_BY
|
||||
|
||||
`ORDER BY` — The sorting key.
|
||||
|
||||
A tuple of column names or arbitrary expressions. Example: `ORDER BY (CounterID, EventDate)`.
|
||||
|
||||
@ -70,19 +74,27 @@ For a description of parameters, see the [CREATE query description](../../../sql
|
||||
|
||||
Use the `ORDER BY tuple()` syntax, if you do not need sorting. See [Selecting the Primary Key](#selecting-the-primary-key).
|
||||
|
||||
- `PARTITION BY` — The [partitioning key](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Optional. In most cases you don't need partition key, and in most other cases you don't need partition key more granular than by months. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead make client identifier or name the first column in the ORDER BY expression).
|
||||
#### PARTITION BY
|
||||
|
||||
`PARTITION BY` — The [partitioning key](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Optional. In most cases you don't need partition key, and in most other cases you don't need partition key more granular than by months. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead make client identifier or name the first column in the ORDER BY expression).
|
||||
|
||||
For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](../../../sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format.
|
||||
|
||||
- `PRIMARY KEY` — The primary key if it [differs from the sorting key](#choosing-a-primary-key-that-differs-from-the-sorting-key). Optional.
|
||||
#### PRIMARY KEY
|
||||
|
||||
`PRIMARY KEY` — The primary key if it [differs from the sorting key](#choosing-a-primary-key-that-differs-from-the-sorting-key). Optional.
|
||||
|
||||
By default the primary key is the same as the sorting key (which is specified by the `ORDER BY` clause). Thus in most cases it is unnecessary to specify a separate `PRIMARY KEY` clause.
|
||||
|
||||
- `SAMPLE BY` — An expression for sampling. Optional.
|
||||
#### SAMPLE BY
|
||||
|
||||
`SAMPLE BY` — An expression for sampling. Optional.
|
||||
|
||||
If a sampling expression is used, the primary key must contain it. The result of a sampling expression must be an unsigned integer. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`.
|
||||
|
||||
- `TTL` — A list of rules specifying storage duration of rows and defining logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). Optional.
|
||||
#### TTL
|
||||
|
||||
`TTL` — A list of rules specifying storage duration of rows and defining logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). Optional.
|
||||
|
||||
Expression must have one `Date` or `DateTime` column as a result. Example:
|
||||
`TTL date + INTERVAL 1 DAY`
|
||||
@ -91,26 +103,76 @@ For a description of parameters, see the [CREATE query description](../../../sql
|
||||
|
||||
For more details, see [TTL for columns and tables](#table_engine-mergetree-ttl)
|
||||
|
||||
- `SETTINGS` — Additional parameters that control the behavior of the `MergeTree` (optional):
|
||||
### SETTINGS
|
||||
Additional parameters that control the behavior of the `MergeTree` (optional):
|
||||
|
||||
- `index_granularity` — Maximum number of data rows between the marks of an index. Default value: 8192. See [Data Storage](#mergetree-data-storage).
|
||||
- `index_granularity_bytes` — Maximum size of data granules in bytes. Default value: 10Mb. To restrict the granule size only by number of rows, set to 0 (not recommended). See [Data Storage](#mergetree-data-storage).
|
||||
- `min_index_granularity_bytes` — Min allowed size of data granules in bytes. Default value: 1024b. To provide a safeguard against accidentally creating tables with very low index_granularity_bytes. See [Data Storage](#mergetree-data-storage).
|
||||
- `enable_mixed_granularity_parts` — Enables or disables transitioning to control the granule size with the `index_granularity_bytes` setting. Before version 19.11, there was only the `index_granularity` setting for restricting granule size. The `index_granularity_bytes` setting improves ClickHouse performance when selecting data from tables with big rows (tens and hundreds of megabytes). If you have tables with big rows, you can enable this setting for the tables to improve the efficiency of `SELECT` queries.
|
||||
- `use_minimalistic_part_header_in_zookeeper` — Storage method of the data parts headers in ZooKeeper. If `use_minimalistic_part_header_in_zookeeper=1`, then ZooKeeper stores less data. For more information, see the [setting description](../../../operations/server-configuration-parameters/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) in “Server configuration parameters”.
|
||||
- `min_merge_bytes_to_use_direct_io` — The minimum data volume for merge operation that is required for using direct I/O access to the storage disk. When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged. If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse reads and writes the data to the storage disk using the direct I/O interface (`O_DIRECT` option). If `min_merge_bytes_to_use_direct_io = 0`, then direct I/O is disabled. Default value: `10 * 1024 * 1024 * 1024` bytes.
|
||||
#### index_granularity
|
||||
|
||||
`index_granularity` — Maximum number of data rows between the marks of an index. Default value: 8192. See [Data Storage](#mergetree-data-storage).
|
||||
|
||||
#### index_granularity_bytes
|
||||
|
||||
`index_granularity_bytes` — Maximum size of data granules in bytes. Default value: 10Mb. To restrict the granule size only by number of rows, set to 0 (not recommended). See [Data Storage](#mergetree-data-storage).
|
||||
|
||||
#### min_index_granularity_bytes
|
||||
|
||||
`min_index_granularity_bytes` — Min allowed size of data granules in bytes. Default value: 1024b. To provide a safeguard against accidentally creating tables with very low index_granularity_bytes. See [Data Storage](#mergetree-data-storage).
|
||||
|
||||
#### enable_mixed_granularity_parts
|
||||
|
||||
`enable_mixed_granularity_parts` — Enables or disables transitioning to control the granule size with the `index_granularity_bytes` setting. Before version 19.11, there was only the `index_granularity` setting for restricting granule size. The `index_granularity_bytes` setting improves ClickHouse performance when selecting data from tables with big rows (tens and hundreds of megabytes). If you have tables with big rows, you can enable this setting for the tables to improve the efficiency of `SELECT` queries.
|
||||
|
||||
#### use_minimalistic_part_header_in_zookeeper
|
||||
|
||||
`use_minimalistic_part_header_in_zookeeper` — Storage method of the data parts headers in ZooKeeper. If `use_minimalistic_part_header_in_zookeeper=1`, then ZooKeeper stores less data. For more information, see the [setting description](../../../operations/server-configuration-parameters/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) in “Server configuration parameters”.
|
||||
|
||||
#### min_merge_bytes_to_use_direct_io
|
||||
|
||||
`min_merge_bytes_to_use_direct_io` — The minimum data volume for merge operation that is required for using direct I/O access to the storage disk. When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged. If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse reads and writes the data to the storage disk using the direct I/O interface (`O_DIRECT` option). If `min_merge_bytes_to_use_direct_io = 0`, then direct I/O is disabled. Default value: `10 * 1024 * 1024 * 1024` bytes.
|
||||
<a name="mergetree_setting-merge_with_ttl_timeout"></a>
|
||||
- `merge_with_ttl_timeout` — Minimum delay in seconds before repeating a merge with delete TTL. Default value: `14400` seconds (4 hours).
|
||||
- `merge_with_recompression_ttl_timeout` — Minimum delay in seconds before repeating a merge with recompression TTL. Default value: `14400` seconds (4 hours).
|
||||
- `try_fetch_recompressed_part_timeout` — Timeout (in seconds) before starting merge with recompression. During this time ClickHouse tries to fetch recompressed part from replica which assigned this merge with recompression. Default value: `7200` seconds (2 hours).
|
||||
- `write_final_mark` — Enables or disables writing the final index mark at the end of data part (after the last byte). Default value: 1. Don’t turn it off.
|
||||
- `merge_max_block_size` — Maximum number of rows in block for merge operations. Default value: 8192.
|
||||
- `storage_policy` — Storage policy. See [Using Multiple Block Devices for Data Storage](#table_engine-mergetree-multiple-volumes).
|
||||
- `min_bytes_for_wide_part`, `min_rows_for_wide_part` — Minimum number of bytes/rows in a data part that can be stored in `Wide` format. You can set one, both or none of these settings. See [Data Storage](#mergetree-data-storage).
|
||||
- `max_parts_in_total` — Maximum number of parts in all partitions.
|
||||
- `max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. You can also specify this setting in the global settings (see [max_compress_block_size](../../../operations/settings/settings.md#max-compress-block-size) setting). The value specified when table is created overrides the global value for this setting.
|
||||
- `min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. You can also specify this setting in the global settings (see [min_compress_block_size](../../../operations/settings/settings.md#min-compress-block-size) setting). The value specified when table is created overrides the global value for this setting.
|
||||
- `max_partitions_to_read` — Limits the maximum number of partitions that can be accessed in one query. You can also specify setting [max_partitions_to_read](../../../operations/settings/merge-tree-settings.md#max-partitions-to-read) in the global setting.
|
||||
|
||||
#### merge_with_ttl_timeout
|
||||
|
||||
`merge_with_ttl_timeout` — Minimum delay in seconds before repeating a merge with delete TTL. Default value: `14400` seconds (4 hours).
|
||||
#### merge_with_recompression_ttl_timeout
|
||||
|
||||
`merge_with_recompression_ttl_timeout` — Minimum delay in seconds before repeating a merge with recompression TTL. Default value: `14400` seconds (4 hours).
|
||||
|
||||
#### try_fetch_recompressed_part_timeout
|
||||
|
||||
`try_fetch_recompressed_part_timeout` — Timeout (in seconds) before starting merge with recompression. During this time ClickHouse tries to fetch recompressed part from replica which assigned this merge with recompression. Default value: `7200` seconds (2 hours).
|
||||
|
||||
#### write_final_mark
|
||||
|
||||
`write_final_mark` — Enables or disables writing the final index mark at the end of data part (after the last byte). Default value: 1. Don’t turn it off.
|
||||
|
||||
#### merge_max_block_size
|
||||
|
||||
`merge_max_block_size` — Maximum number of rows in block for merge operations. Default value: 8192.
|
||||
|
||||
#### storage_policy
|
||||
|
||||
`storage_policy` — Storage policy. See [Using Multiple Block Devices for Data Storage](#table_engine-mergetree-multiple-volumes).
|
||||
|
||||
#### min_bytes_for_wide_part
|
||||
|
||||
`min_bytes_for_wide_part`, `min_rows_for_wide_part` — Minimum number of bytes/rows in a data part that can be stored in `Wide` format. You can set one, both or none of these settings. See [Data Storage](#mergetree-data-storage).
|
||||
|
||||
#### max_parts_in_total
|
||||
|
||||
`max_parts_in_total` — Maximum number of parts in all partitions.
|
||||
|
||||
#### max_compress_block_size
|
||||
|
||||
`max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. You can also specify this setting in the global settings (see [max_compress_block_size](../../../operations/settings/settings.md#max-compress-block-size) setting). The value specified when table is created overrides the global value for this setting.
|
||||
|
||||
#### min_compress_block_size
|
||||
|
||||
`min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. You can also specify this setting in the global settings (see [min_compress_block_size](../../../operations/settings/settings.md#min-compress-block-size) setting). The value specified when table is created overrides the global value for this setting.
|
||||
|
||||
#### max_partitions_to_read
|
||||
|
||||
`max_partitions_to_read` — Limits the maximum number of partitions that can be accessed in one query. You can also specify setting [max_partitions_to_read](../../../operations/settings/merge-tree-settings.md#max-partitions-to-read) in the global setting.
|
||||
|
||||
**Example of Sections Setting**
|
||||
|
||||
@ -310,17 +372,17 @@ SELECT count() FROM table WHERE s < 'z'
|
||||
SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234
|
||||
```
|
||||
|
||||
#### Available Types of Indices {#available-types-of-indices}
|
||||
### Available Types of Indices {#available-types-of-indices}
|
||||
|
||||
- `minmax`
|
||||
#### `minmax`
|
||||
|
||||
Stores extremes of the specified expression (if the expression is `tuple`, then it stores extremes for each element of `tuple`), uses stored info for skipping blocks of data like the primary key.
|
||||
|
||||
- `set(max_rows)`
|
||||
#### `set(max_rows)`
|
||||
|
||||
Stores unique values of the specified expression (no more than `max_rows` rows, `max_rows=0` means “no limits”). Uses the values to check if the `WHERE` expression is not satisfiable on a block of data.
|
||||
|
||||
- `ngrambf_v1(n, size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)`
|
||||
#### `ngrambf_v1(n, size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)`
|
||||
|
||||
Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) that contains all ngrams from a block of data. Works only with datatypes: [String](../../../sql-reference/data-types/string.md), [FixedString](../../../sql-reference/data-types/fixedstring.md) and [Map](../../../sql-reference/data-types/map.md). Can be used for optimization of `EQUALS`, `LIKE` and `IN` expressions.
|
||||
|
||||
@ -329,11 +391,11 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234
|
||||
- `number_of_hash_functions` — The number of hash functions used in the Bloom filter.
|
||||
- `random_seed` — The seed for Bloom filter hash functions.
|
||||
|
||||
- `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)`
|
||||
#### `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)`
|
||||
|
||||
The same as `ngrambf_v1`, but stores tokens instead of ngrams. Tokens are sequences separated by non-alphanumeric characters.
|
||||
|
||||
- `bloom_filter([false_positive])` — Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) for the specified columns.
|
||||
#### `bloom_filter([false_positive])` — Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) for the specified columns.
|
||||
|
||||
The optional `false_positive` parameter is the probability of receiving a false positive response from the filter. Possible values: (0, 1). Default value: 0.025.
|
||||
|
||||
@ -357,7 +419,7 @@ INDEX sample_index2 (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100
|
||||
INDEX sample_index3 (lower(str), str) TYPE ngrambf_v1(3, 256, 2, 0) GRANULARITY 4
|
||||
```
|
||||
|
||||
#### Functions Support {#functions-support}
|
||||
### Functions Support {#functions-support}
|
||||
|
||||
Conditions in the `WHERE` clause contains calls of the functions that operate with columns. If the column is a part of an index, ClickHouse tries to use this index when performing the functions. ClickHouse supports different subsets of functions for using indexes.
|
||||
|
||||
@ -466,7 +528,7 @@ The `TTL` clause can’t be used for key columns.
|
||||
|
||||
**Examples**
|
||||
|
||||
Creating a table with `TTL`:
|
||||
#### Creating a table with `TTL`:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE example_table
|
||||
@ -481,7 +543,7 @@ PARTITION BY toYYYYMM(d)
|
||||
ORDER BY d;
|
||||
```
|
||||
|
||||
Adding TTL to a column of an existing table
|
||||
#### Adding TTL to a column of an existing table
|
||||
|
||||
``` sql
|
||||
ALTER TABLE example_table
|
||||
@ -489,7 +551,7 @@ ALTER TABLE example_table
|
||||
c String TTL d + INTERVAL 1 DAY;
|
||||
```
|
||||
|
||||
Altering TTL of the column
|
||||
#### Altering TTL of the column
|
||||
|
||||
``` sql
|
||||
ALTER TABLE example_table
|
||||
@ -524,7 +586,7 @@ If a column is not part of the `GROUP BY` expression and is not set explicitly i
|
||||
|
||||
**Examples**
|
||||
|
||||
Creating a table with `TTL`:
|
||||
#### Creating a table with `TTL`:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE example_table
|
||||
@ -540,7 +602,7 @@ TTL d + INTERVAL 1 MONTH [DELETE],
|
||||
d + INTERVAL 2 WEEK TO DISK 'bbb';
|
||||
```
|
||||
|
||||
Altering `TTL` of the table:
|
||||
#### Altering `TTL` of the table:
|
||||
|
||||
``` sql
|
||||
ALTER TABLE example_table
|
||||
@ -561,7 +623,7 @@ ORDER BY d
|
||||
TTL d + INTERVAL 1 MONTH DELETE WHERE toDayOfWeek(d) = 1;
|
||||
```
|
||||
|
||||
Creating a table, where expired rows are recompressed:
|
||||
#### Creating a table, where expired rows are recompressed:
|
||||
|
||||
```sql
|
||||
CREATE TABLE table_for_recompression
|
||||
|
@ -33,16 +33,18 @@ For a description of request parameters, see [statement description](../../../sq
|
||||
Uniqueness of rows is determined by the `ORDER BY` table section, not `PRIMARY KEY`.
|
||||
:::
|
||||
|
||||
**ReplacingMergeTree Parameters**
|
||||
## ReplacingMergeTree Parameters
|
||||
|
||||
- `ver` — column with the version number. Type `UInt*`, `Date`, `DateTime` or `DateTime64`. Optional parameter.
|
||||
### ver
|
||||
|
||||
`ver` — column with the version number. Type `UInt*`, `Date`, `DateTime` or `DateTime64`. Optional parameter.
|
||||
|
||||
When merging, `ReplacingMergeTree` from all the rows with the same sorting key leaves only one:
|
||||
|
||||
- The last in the selection, if `ver` not set. A selection is a set of rows in a set of parts participating in the merge. The most recently created part (the last insert) will be the last one in the selection. Thus, after deduplication, the very last row from the most recent insert will remain for each unique sorting key.
|
||||
- With the maximum version, if `ver` specified. If `ver` is the same for several rows, then it will use "if `ver` is not specified" rule for them, i.e. the most recent inserted row will remain.
|
||||
|
||||
**Query clauses**
|
||||
## Query clauses
|
||||
|
||||
When creating a `ReplacingMergeTree` table the same [clauses](../../../engines/table-engines/mergetree-family/mergetree.md) are required, as when creating a `MergeTree` table.
|
||||
|
||||
|
@ -120,11 +120,19 @@ The system monitors data synchronicity on replicas and is able to recover after
|
||||
|
||||
The `Replicated` prefix is added to the table engine name. For example:`ReplicatedMergeTree`.
|
||||
|
||||
**Replicated\*MergeTree parameters**
|
||||
### Replicated\*MergeTree parameters
|
||||
|
||||
- `zoo_path` — The path to the table in ClickHouse Keeper.
|
||||
- `replica_name` — The replica name in ClickHouse Keeper.
|
||||
- `other_parameters` — Parameters of an engine which is used for creating the replicated version, for example, version in `ReplacingMergeTree`.
|
||||
#### zoo_path
|
||||
|
||||
`zoo_path` — The path to the table in ClickHouse Keeper.
|
||||
|
||||
#### replica_name
|
||||
|
||||
`replica_name` — The replica name in ClickHouse Keeper.
|
||||
|
||||
#### other_parameters
|
||||
|
||||
`other_parameters` — Parameters of an engine which is used for creating the replicated version, for example, version in `ReplacingMergeTree`.
|
||||
|
||||
Example:
|
||||
|
||||
|
@ -26,14 +26,16 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
|
||||
For a description of request parameters, see [request description](../../../sql-reference/statements/create/table.md).
|
||||
|
||||
**Parameters of SummingMergeTree**
|
||||
### Parameters of SummingMergeTree
|
||||
|
||||
- `columns` - a tuple with the names of columns where values will be summarized. Optional parameter.
|
||||
#### columns
|
||||
|
||||
`columns` - a tuple with the names of columns where values will be summarized. Optional parameter.
|
||||
The columns must be of a numeric type and must not be in the primary key.
|
||||
|
||||
If `columns` not specified, ClickHouse summarizes the values in all columns with a numeric data type that are not in the primary key.
|
||||
|
||||
**Query clauses**
|
||||
### Query clauses
|
||||
|
||||
When creating a `SummingMergeTree` table the same [clauses](../../../engines/table-engines/mergetree-family/mergetree.md) are required, as when creating a `MergeTree` table.
|
||||
|
||||
|
@ -31,21 +31,25 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
|
||||
For a description of query parameters, see the [query description](../../../sql-reference/statements/create/table.md).
|
||||
|
||||
**Engine Parameters**
|
||||
### Engine Parameters
|
||||
|
||||
``` sql
|
||||
VersionedCollapsingMergeTree(sign, version)
|
||||
```
|
||||
|
||||
- `sign` — Name of the column with the type of row: `1` is a “state” row, `-1` is a “cancel” row.
|
||||
#### sign
|
||||
|
||||
`sign` — Name of the column with the type of row: `1` is a “state” row, `-1` is a “cancel” row.
|
||||
|
||||
The column data type should be `Int8`.
|
||||
|
||||
- `version` — Name of the column with the version of the object state.
|
||||
#### version
|
||||
|
||||
`version` — Name of the column with the version of the object state.
|
||||
|
||||
The column data type should be `UInt*`.
|
||||
|
||||
**Query Clauses**
|
||||
### Query Clauses
|
||||
|
||||
When creating a `VersionedCollapsingMergeTree` table, the same [clauses](../../../engines/table-engines/mergetree-family/mergetree.md) are required as when creating a `MergeTree` table.
|
||||
|
||||
|
@ -11,24 +11,45 @@ Buffers the data to write in RAM, periodically flushing it to another table. Dur
|
||||
Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes)
|
||||
```
|
||||
|
||||
Engine parameters:
|
||||
### Engine parameters:
|
||||
|
||||
- `database` – Database name. You can use `currentDatabase()` or another constant expression that returns a string.
|
||||
- `table` – Table to flush data to.
|
||||
- `num_layers` – Parallelism layer. Physically, the table will be represented as `num_layers` of independent buffers. Recommended value: 16.
|
||||
- `min_time`, `max_time`, `min_rows`, `max_rows`, `min_bytes`, and `max_bytes` – Conditions for flushing data from the buffer.
|
||||
#### database
|
||||
|
||||
Optional engine parameters:
|
||||
`database` – Database name. You can use `currentDatabase()` or another constant expression that returns a string.
|
||||
|
||||
- `flush_time`, `flush_rows`, `flush_bytes` – Conditions for flushing data from the buffer, that will happen only in background (omitted or zero means no `flush*` parameters).
|
||||
#### table
|
||||
|
||||
`table` – Table to flush data to.
|
||||
|
||||
#### num_layers
|
||||
|
||||
`num_layers` – Parallelism layer. Physically, the table will be represented as `num_layers` of independent buffers. Recommended value: 16.
|
||||
|
||||
#### min_time, max_time, min_rows, max_rows, min_bytes, and max_bytes
|
||||
|
||||
Conditions for flushing data from the buffer.
|
||||
|
||||
### Optional engine parameters:
|
||||
|
||||
#### flush_time, flush_rows, and flush_bytes
|
||||
|
||||
Conditions for flushing data from the buffer, that will happen only in background (omitted or zero means no `flush*` parameters).
|
||||
|
||||
Data is flushed from the buffer and written to the destination table if all the `min*` conditions or at least one `max*` condition are met.
|
||||
|
||||
Also, if at least one `flush*` condition are met flush initiated in background, this is different from `max*`, since `flush*` allows you to configure background flushes separately to avoid adding latency for `INSERT` (into `Buffer`) queries.
|
||||
|
||||
- `min_time`, `max_time`, `flush_time` – Condition for the time in seconds from the moment of the first write to the buffer.
|
||||
- `min_rows`, `max_rows`, `flush_rows` – Condition for the number of rows in the buffer.
|
||||
- `min_bytes`, `max_bytes`, `flush_bytes` – Condition for the number of bytes in the buffer.
|
||||
#### min_time, max_time, and flush_time
|
||||
|
||||
Condition for the time in seconds from the moment of the first write to the buffer.
|
||||
|
||||
#### min_rows, max_rows, and flush_rows
|
||||
|
||||
Condition for the number of rows in the buffer.
|
||||
|
||||
#### min_bytes, max_bytes, and flush_bytes
|
||||
|
||||
Condition for the number of bytes in the buffer.
|
||||
|
||||
During the write operation, data is inserted to a `num_layers` number of random buffers. Or, if the data part to insert is large enough (greater than `max_rows` or `max_bytes`), it is written directly to the destination table, omitting the buffer.
|
||||
|
||||
|
@ -27,42 +27,70 @@ When the `Distributed` table is pointing to a table on the current server you ca
|
||||
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] AS [db2.]name2 ENGINE = Distributed(cluster, database, table[, sharding_key[, policy_name]]) [SETTINGS name=value, ...]
|
||||
```
|
||||
|
||||
**Distributed Parameters**
|
||||
### Distributed Parameters
|
||||
|
||||
- `cluster` - the cluster name in the server’s config file
|
||||
#### cluster
|
||||
|
||||
- `database` - the name of a remote database
|
||||
`cluster` - the cluster name in the server’s config file
|
||||
|
||||
- `table` - the name of a remote table
|
||||
#### database
|
||||
|
||||
- `sharding_key` - (optionally) sharding key
|
||||
`database` - the name of a remote database
|
||||
|
||||
- `policy_name` - (optionally) policy name, it will be used to store temporary files for async send
|
||||
#### table
|
||||
|
||||
`table` - the name of a remote table
|
||||
|
||||
#### sharding_key
|
||||
|
||||
`sharding_key` - (optionally) sharding key
|
||||
|
||||
#### policy_name
|
||||
|
||||
`policy_name` - (optionally) policy name, it will be used to store temporary files for async send
|
||||
|
||||
**See Also**
|
||||
|
||||
- [insert_distributed_sync](../../../operations/settings/settings.md#insert_distributed_sync) setting
|
||||
- [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) for the examples
|
||||
|
||||
**Distributed Settings**
|
||||
### Distributed Settings
|
||||
|
||||
- `fsync_after_insert` - do the `fsync` for the file data after asynchronous insert to Distributed. Guarantees that the OS flushed the whole inserted data to a file **on the initiator node** disk.
|
||||
#### fsync_after_insert
|
||||
|
||||
- `fsync_directories` - do the `fsync` for directories. Guarantees that the OS refreshed directory metadata after operations related to asynchronous inserts on Distributed table (after insert, after sending the data to shard, etc).
|
||||
`fsync_after_insert` - do the `fsync` for the file data after asynchronous insert to Distributed. Guarantees that the OS flushed the whole inserted data to a file **on the initiator node** disk.
|
||||
|
||||
- `bytes_to_throw_insert` - if more than this number of compressed bytes will be pending for async INSERT, an exception will be thrown. 0 - do not throw. Default 0.
|
||||
#### fsync_directories
|
||||
|
||||
- `bytes_to_delay_insert` - if more than this number of compressed bytes will be pending for async INSERT, the query will be delayed. 0 - do not delay. Default 0.
|
||||
`fsync_directories` - do the `fsync` for directories. Guarantees that the OS refreshed directory metadata after operations related to asynchronous inserts on Distributed table (after insert, after sending the data to shard, etc).
|
||||
|
||||
- `max_delay_to_insert` - max delay of inserting data into Distributed table in seconds, if there are a lot of pending bytes for async send. Default 60.
|
||||
#### bytes_to_throw_insert
|
||||
|
||||
- `monitor_batch_inserts` - same as [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts)
|
||||
`bytes_to_throw_insert` - if more than this number of compressed bytes will be pending for async INSERT, an exception will be thrown. 0 - do not throw. Default 0.
|
||||
|
||||
- `monitor_split_batch_on_failure` - same as [distributed_directory_monitor_split_batch_on_failure](../../../operations/settings/settings.md#distributed_directory_monitor_split_batch_on_failure)
|
||||
#### bytes_to_delay_insert
|
||||
|
||||
- `monitor_sleep_time_ms` - same as [distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms)
|
||||
`bytes_to_delay_insert` - if more than this number of compressed bytes will be pending for async INSERT, the query will be delayed. 0 - do not delay. Default 0.
|
||||
|
||||
- `monitor_max_sleep_time_ms` - same as [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms)
|
||||
#### max_delay_to_insert
|
||||
|
||||
`max_delay_to_insert` - max delay of inserting data into Distributed table in seconds, if there are a lot of pending bytes for async send. Default 60.
|
||||
|
||||
#### monitor_batch_inserts
|
||||
|
||||
`monitor_batch_inserts` - same as [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts)
|
||||
|
||||
#### monitor_split_batch_on_failure
|
||||
|
||||
`monitor_split_batch_on_failure` - same as [distributed_directory_monitor_split_batch_on_failure](../../../operations/settings/settings.md#distributed_directory_monitor_split_batch_on_failure)
|
||||
|
||||
#### monitor_sleep_time_ms
|
||||
|
||||
`monitor_sleep_time_ms` - same as [distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms)
|
||||
|
||||
#### monitor_max_sleep_time_ms
|
||||
|
||||
`monitor_max_sleep_time_ms` - same as [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms)
|
||||
|
||||
:::note
|
||||
**Durability settings** (`fsync_...`):
|
||||
@ -213,7 +241,9 @@ To learn more about how distibuted `in` and `global in` queries are processed, r
|
||||
|
||||
## Virtual Columns {#virtual-columns}
|
||||
|
||||
- `_shard_num` — Contains the `shard_num` value from the table `system.clusters`. Type: [UInt32](../../../sql-reference/data-types/int-uint.md).
|
||||
#### _shard_num
|
||||
|
||||
`_shard_num` — Contains the `shard_num` value from the table `system.clusters`. Type: [UInt32](../../../sql-reference/data-types/int-uint.md).
|
||||
|
||||
:::note
|
||||
Since [remote](../../../sql-reference/table-functions/remote.md) and [cluster](../../../sql-reference/table-functions/cluster.md) table functions internally create temporary Distributed table, `_shard_num` is available there too.
|
||||
|
@ -23,11 +23,19 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
|
||||
See the detailed description of the [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query) query.
|
||||
|
||||
**Engine Parameters**
|
||||
## Engine Parameters
|
||||
|
||||
- `join_strictness` – [JOIN strictness](../../../sql-reference/statements/select/join.md#select-join-types).
|
||||
- `join_type` – [JOIN type](../../../sql-reference/statements/select/join.md#select-join-types).
|
||||
- `k1[, k2, ...]` – Key columns from the `USING` clause that the `JOIN` operation is made with.
|
||||
### join_strictness
|
||||
|
||||
`join_strictness` – [JOIN strictness](../../../sql-reference/statements/select/join.md#select-join-types).
|
||||
|
||||
### join_type
|
||||
|
||||
`join_type` – [JOIN type](../../../sql-reference/statements/select/join.md#select-join-types).
|
||||
|
||||
### Key columns
|
||||
|
||||
`k1[, k2, ...]` – Key columns from the `USING` clause that the `JOIN` operation is made with.
|
||||
|
||||
Enter `join_strictness` and `join_type` parameters without quotes, for example, `Join(ANY, LEFT, col1)`. They must match the `JOIN` operation that the table will be used for. If the parameters do not match, ClickHouse does not throw an exception and may return incorrect data.
|
||||
|
||||
@ -56,12 +64,28 @@ Main use-cases for `Join`-engine tables are following:
|
||||
|
||||
When creating a table, the following settings are applied:
|
||||
|
||||
- [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls)
|
||||
- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join)
|
||||
- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join)
|
||||
- [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode)
|
||||
- [join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row)
|
||||
- [persistent](../../../operations/settings/settings.md#persistent)
|
||||
#### join_use_nulls
|
||||
|
||||
[join_use_nulls](../../../operations/settings/settings.md#join_use_nulls)
|
||||
|
||||
#### max_rows_in_join
|
||||
|
||||
[max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join)
|
||||
|
||||
#### max_bytes_in_join
|
||||
|
||||
[max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join)
|
||||
|
||||
#### join_overflow_mode
|
||||
|
||||
[join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode)
|
||||
|
||||
#### join_any_take_last_row
|
||||
|
||||
[join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row)
|
||||
#### join_use_nulls
|
||||
|
||||
[persistent](../../../operations/settings/settings.md#persistent)
|
||||
|
||||
The `Join`-engine tables can’t be used in `GLOBAL JOIN` operations.
|
||||
|
||||
|
@ -15,14 +15,18 @@ Reading is automatically parallelized. Writing to a table is not supported. When
|
||||
CREATE TABLE ... Engine=Merge(db_name, tables_regexp)
|
||||
```
|
||||
|
||||
**Engine Parameters**
|
||||
## Engine Parameters
|
||||
|
||||
- `db_name` — Possible values:
|
||||
### db_name
|
||||
|
||||
`db_name` — Possible values:
|
||||
- database name,
|
||||
- constant expression that returns a string with a database name, for example, `currentDatabase()`,
|
||||
- `REGEXP(expression)`, where `expression` is a regular expression to match the DB names.
|
||||
|
||||
- `tables_regexp` — A regular expression to match the table names in the specified DB or DBs.
|
||||
### tables_regexp
|
||||
|
||||
`tables_regexp` — A regular expression to match the table names in the specified DB or DBs.
|
||||
|
||||
Regular expressions — [re2](https://github.com/google/re2) (supports a subset of PCRE), case-sensitive.
|
||||
See the notes about escaping symbols in regular expressions in the "match" section.
|
||||
|
@ -1,10 +1,9 @@
|
||||
---
|
||||
sidebar_position: 48
|
||||
sidebar_label: Access Control and Account Management
|
||||
title: Access Control and Account Management
|
||||
---
|
||||
|
||||
# Access Control and Account Management
|
||||
|
||||
ClickHouse supports access control management based on [RBAC](https://en.wikipedia.org/wiki/Role-based_access_control) approach.
|
||||
|
||||
ClickHouse access entities:
|
||||
|
@ -494,6 +494,48 @@ If the ‘s’ string is non-empty and does not contain the ‘c’ character at
|
||||
|
||||
Returns the string ‘s’ that was converted from the encoding in ‘from’ to the encoding in ‘to’.
|
||||
|
||||
## Base58Encode(plaintext[, alphabet_name]), Base58Decode(encoded_text[, alphabet_name])
|
||||
|
||||
Accepts a String and encodes/decodes it using [Base58](https://tools.ietf.org/id/draft-msporny-base58-01.html) encoding scheme using specified alphabet.
|
||||
|
||||
**Syntax**
|
||||
|
||||
```sql
|
||||
base58Encode(decoded[, alphabet_name])
|
||||
base58Decode(encoded[, alphabet_name])
|
||||
```
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `decoded` — [String](../../sql-reference/data-types/string.md) column or constant.
|
||||
- `encoded` — [String](../../sql-reference/data-types/string.md) column or constant. If the string is not a valid base58-encoded value, an exception is thrown.
|
||||
- `alphabet_name` — String constant. Specifies alphabet used for encoding. Possible values: `gmp`, `bitcoin`, `ripple`, `flickr`. Default: `bitcoin`.
|
||||
|
||||
**Returned value**
|
||||
|
||||
- A string containing encoded/decoded value of 1st argument.
|
||||
|
||||
Type: [String](../../sql-reference/data-types/string.md).
|
||||
|
||||
**Example**
|
||||
|
||||
Query:
|
||||
|
||||
``` sql
|
||||
SELECT base58Encode('encode', 'flickr');
|
||||
SELECT base58Decode('izCFiDUY', 'ripple');
|
||||
```
|
||||
|
||||
Result:
|
||||
```text
|
||||
┌─base58Encode('encode', 'flickr')─┐
|
||||
│ SvyTHb1D │
|
||||
└──────────────────────────────────┘
|
||||
┌─base58Decode('izCFiDUY', 'ripple')─┐
|
||||
│ decode │
|
||||
└────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## base64Encode(s)
|
||||
|
||||
Encodes ‘s’ string into base64
|
||||
|
@ -18,13 +18,13 @@ Each action is an operation on a column.
|
||||
|
||||
The following actions are supported:
|
||||
|
||||
- [ADD COLUMN](#alter_add-column) — Adds a new column to the table.
|
||||
- [DROP COLUMN](#alter_drop-column) — Deletes the column.
|
||||
- [RENAME COLUMN](#alter_rename-column) — Renames an existing column.
|
||||
- [CLEAR COLUMN](#alter_clear-column) — Resets column values.
|
||||
- [COMMENT COLUMN](#alter_comment-column) — Adds a text comment to the column.
|
||||
- [MODIFY COLUMN](#alter_modify-column) — Changes column’s type, default expression and TTL.
|
||||
- [MODIFY COLUMN REMOVE](#modify-remove) — Removes one of the column properties.
|
||||
- [ADD COLUMN](#add-column) — Adds a new column to the table.
|
||||
- [DROP COLUMN](#drop-column) — Deletes the column.
|
||||
- [RENAME COLUMN](#rename-column) — Renames an existing column.
|
||||
- [CLEAR COLUMN](#clear-column) — Resets column values.
|
||||
- [COMMENT COLUMN](#comment-column) — Adds a text comment to the column.
|
||||
- [MODIFY COLUMN](#modify-column) — Changes column’s type, default expression and TTL.
|
||||
- [MODIFY COLUMN REMOVE](#modify-column-remove) — Removes one of the column properties.
|
||||
- [MATERIALIZE COLUMN](#materialize-column) — Materializes the column in the parts where the column is missing.
|
||||
|
||||
These actions are described in detail below.
|
||||
|
@ -7,18 +7,18 @@ sidebar_label: PARTITION
|
||||
|
||||
The following operations with [partitions](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md) are available:
|
||||
|
||||
- [DETACH PARTITION](#alter_detach-partition) — Moves a partition to the `detached` directory and forget it.
|
||||
- [DROP PARTITION](#alter_drop-partition) — Deletes a partition.
|
||||
- [ATTACH PART\|PARTITION](#alter_attach-partition) — Adds a part or partition from the `detached` directory to the table.
|
||||
- [ATTACH PARTITION FROM](#alter_attach-partition-from) — Copies the data partition from one table to another and adds.
|
||||
- [REPLACE PARTITION](#alter_replace-partition) — Copies the data partition from one table to another and replaces.
|
||||
- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition) — Moves the data partition from one table to another.
|
||||
- [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) — Resets the value of a specified column in a partition.
|
||||
- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) — Resets the specified secondary index in a partition.
|
||||
- [FREEZE PARTITION](#alter_freeze-partition) — Creates a backup of a partition.
|
||||
- [UNFREEZE PARTITION](#alter_unfreeze-partition) — Removes a backup of a partition.
|
||||
- [FETCH PARTITION\|PART](#alter_fetch-partition) — Downloads a part or partition from another server.
|
||||
- [MOVE PARTITION\|PART](#alter_move-partition) — Move partition/data part to another disk or volume.
|
||||
- [DETACH PARTITION\|PART](#detach-partitionpart) — Moves a partition or part to the `detached` directory and forget it.
|
||||
- [DROP PARTITION\|PART](#drop-partitionpart) — Deletes a partition or part.
|
||||
- [ATTACH PARTITION\|PART](#attach-partitionpart) — Adds a partition or part from the `detached` directory to the table.
|
||||
- [ATTACH PARTITION FROM](#attach-partition-from) — Copies the data partition from one table to another and adds.
|
||||
- [REPLACE PARTITION](#replace-partition) — Copies the data partition from one table to another and replaces.
|
||||
- [MOVE PARTITION TO TABLE](#move-partition-to-table) — Moves the data partition from one table to another.
|
||||
- [CLEAR COLUMN IN PARTITION](#clear-column-in-partition) — Resets the value of a specified column in a partition.
|
||||
- [CLEAR INDEX IN PARTITION](#clear-index-in-partition) — Resets the specified secondary index in a partition.
|
||||
- [FREEZE PARTITION](#freeze-partition) — Creates a backup of a partition.
|
||||
- [UNFREEZE PARTITION](#unfreeze-partition) — Removes a backup of a partition.
|
||||
- [FETCH PARTITION\|PART](#fetch-partitionpart) — Downloads a part or partition from another server.
|
||||
- [MOVE PARTITION\|PART](#move-partitionpart) — Move partition/data part to another disk or volume.
|
||||
- [UPDATE IN PARTITION](#update-in-partition) — Update data inside the partition by condition.
|
||||
- [DELETE IN PARTITION](#delete-in-partition) — Delete data inside the partition by condition.
|
||||
|
||||
|
@ -18,6 +18,7 @@ sidebar_label: "Используемые сторонние библиотеки
|
||||
| aws-c-common | [Apache](https://github.com/ClickHouse-Extras/aws-c-common/blob/736a82d1697c108b04a277e66438a7f4e19b6857/LICENSE) |
|
||||
| aws-c-event-stream | [Apache](https://github.com/ClickHouse-Extras/aws-c-event-stream/blob/3bc33662f9ccff4f4cbcf9509cc78c26e022fde0/LICENSE) |
|
||||
| aws-checksums | [Apache](https://github.com/ClickHouse-Extras/aws-checksums/blob/519d6d9093819b6cf89ffff589a27ef8f83d0f65/LICENSE) |
|
||||
| base58 | [MIT](https://github.com/ClickHouse/base-x/blob/3e58874643c087f57e82b0ff03825c933fab945a/LICENSE) |
|
||||
| base64 | [BSD 2-clause](https://github.com/ClickHouse-Extras/Turbo-Base64/blob/af9b331f2b4f30b41c70f3a571ff904a8251c1d3/LICENSE) |
|
||||
| boost | [Boost](https://github.com/ClickHouse-Extras/boost/blob/9cf09dbfd55a5c6202dedbdf40781a51b02c2675/LICENSE_1_0.txt) |
|
||||
| boringssl | [BSD](https://github.com/ClickHouse-Extras/boringssl/blob/a6a2e2ab3e44d97ce98e51c558e989f211de7eb3/LICENSE) |
|
||||
|
@ -490,6 +490,48 @@ SELECT concat(key1, key2), sum(value) FROM key_val GROUP BY (key1, key2);
|
||||
|
||||
Возвращает сконвертированную из кодировки from в кодировку to строку s.
|
||||
|
||||
## Base58Encode(plaintext[, alphabet_name]), Base58Decode(plaintext[, alphabet_name]) {#base58}
|
||||
|
||||
Принимает на вход строку или колонку строк и кодирует/раскодирует их с помощью схемы кодирования [Base58](https://tools.ietf.org/id/draft-msporny-base58-01.html) с использованием указанного алфавита.
|
||||
|
||||
**Синтаксис**
|
||||
|
||||
```sql
|
||||
base58Encode(decoded[, alphabet_name])
|
||||
base58Decode(encoded[, alphabet_name])
|
||||
```
|
||||
|
||||
**Аргументы**
|
||||
|
||||
- `decoded` — Колонка или строка типа [String](../../sql-reference/data-types/string.md).
|
||||
- `encoded` — Колонка или строка типа [String](../../sql-reference/data-types/string.md). Если входная строка не является корректным кодом для какой-либо другой строки, возникнет исключение `1001`.
|
||||
- `alphabet_name` — Строковая константа. Указывает алфавит, для которого необходимо получить код. Может принимать одно из следующих значений: `gmp`, `bitcoin`, `ripple`, `flickr`. По умолчанию: `bitcoin`.
|
||||
|
||||
**Возвращаемое значение**
|
||||
|
||||
- Строка, содержащая раскодированный/закодированный первый аргумент.
|
||||
|
||||
Тип: [String](../../sql-reference/data-types/string.md).
|
||||
|
||||
**Пример:**
|
||||
|
||||
Запрос:
|
||||
|
||||
``` sql
|
||||
SELECT base58Encode('encode', 'flickr');
|
||||
SELECT base58Decode('izCFiDUY', 'ripple');
|
||||
```
|
||||
|
||||
Результат:
|
||||
```text
|
||||
┌─base58Encode('encode', 'flickr')─┐
|
||||
│ SvyTHb1D │
|
||||
└──────────────────────────────────┘
|
||||
┌─base58Decode('izCFiDUY', 'ripple')─┐
|
||||
│ decode │
|
||||
└────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## base64Encode(s) {#base64encode}
|
||||
|
||||
Производит кодирование строки s в base64-представление.
|
||||
|
@ -91,7 +91,11 @@ T execute(nanodbc::ConnectionHolderPtr connection_holder, std::function<T(nanodb
|
||||
}
|
||||
catch (const nanodbc::database_error & e)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
LOG_ERROR(
|
||||
&Poco::Logger::get("ODBCConnection"),
|
||||
"ODBC query failed with error: {}, state: {}, native code: {}",
|
||||
e.what(), e.state(), e.native());
|
||||
|
||||
/// SQLState, connection related errors start with 08 (main: 08S01), cursor invalid state is 24000.
|
||||
/// Invalid cursor state is a retriable error.
|
||||
/// Invalid transaction state 25000. Truncate to 2 letters on purpose.
|
||||
|
@ -12,7 +12,6 @@
|
||||
|
||||
<path>./</path>
|
||||
|
||||
<uncompressed_cache_size>8589934592</uncompressed_cache_size>
|
||||
<mlock_executable>true</mlock_executable>
|
||||
|
||||
<users>
|
||||
|
@ -25,6 +25,7 @@ enum class AccessType
|
||||
M(SHOW_DICTIONARIES, "", DICTIONARY, SHOW) /* allows to execute SHOW DICTIONARIES, SHOW CREATE DICTIONARY, EXISTS <dictionary>;
|
||||
implicitly enabled by any grant on the dictionary */\
|
||||
M(SHOW, "", GROUP, ALL) /* allows to execute SHOW, USE, EXISTS, CHECK, DESCRIBE */\
|
||||
M(SHOW_CACHES, "", GROUP, ALL) \
|
||||
\
|
||||
M(SELECT, "", COLUMN, ALL) \
|
||||
M(INSERT, "", COLUMN, ALL) \
|
||||
|
@ -250,9 +250,12 @@ ColumnWithTypeAndName ColumnFunction::reduce() const
|
||||
"arguments but " + toString(captured) + " columns were captured.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
ColumnsWithTypeAndName columns = captured_columns;
|
||||
if (is_short_circuit_argument)
|
||||
IFunction::ShortCircuitSettings settings;
|
||||
/// Arguments of lazy executed function can also be lazy executed.
|
||||
/// But we shouldn't execute arguments if this function is short circuit,
|
||||
/// because it will handle lazy executed arguments by itself.
|
||||
if (is_short_circuit_argument && !function->isShortCircuit(settings, args))
|
||||
{
|
||||
/// Arguments of lazy executed function can also be lazy executed.
|
||||
for (auto & col : columns)
|
||||
{
|
||||
if (const ColumnFunction * arg = checkAndGetShortCircuitArgument(col.column))
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include "FileCacheFactory.h"
|
||||
#include "IFileCache.h"
|
||||
#include "LRUFileCache.h"
|
||||
|
||||
namespace DB
|
||||
@ -19,51 +18,66 @@ FileCacheFactory & FileCacheFactory::instance()
|
||||
FileCacheFactory::CacheByBasePath FileCacheFactory::getAll()
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return caches;
|
||||
return caches_by_path;
|
||||
}
|
||||
|
||||
const FileCacheSettings & FileCacheFactory::getSettings(const std::string & cache_base_path)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = caches_by_path.find(cache_base_path);
|
||||
if (it == caches_by_path.end())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by path: {}", cache_base_path);
|
||||
return it->second->settings;
|
||||
|
||||
auto * cache_data = getImpl(cache_base_path, lock);
|
||||
if (cache_data)
|
||||
return cache_data->settings;
|
||||
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by path: {}", cache_base_path);
|
||||
}
|
||||
|
||||
FileCacheFactory::CacheData * FileCacheFactory::getImpl(const std::string & cache_base_path, std::lock_guard<std::mutex> &)
|
||||
{
|
||||
auto it = caches.find(cache_base_path);
|
||||
if (it == caches.end())
|
||||
return nullptr;
|
||||
return &it->second;
|
||||
}
|
||||
|
||||
FileCachePtr FileCacheFactory::get(const std::string & cache_base_path)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = caches_by_path.find(cache_base_path);
|
||||
if (it == caches_by_path.end())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by path: {}", cache_base_path);
|
||||
return it->second->cache;
|
||||
|
||||
auto * cache_data = getImpl(cache_base_path, lock);
|
||||
if (cache_data)
|
||||
return cache_data->cache;
|
||||
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by path: {}", cache_base_path);
|
||||
}
|
||||
|
||||
FileCachePtr FileCacheFactory::getOrCreate(
|
||||
const std::string & cache_base_path, const FileCacheSettings & file_cache_settings)
|
||||
const std::string & cache_base_path, const FileCacheSettings & file_cache_settings, const std::string & name)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
auto * cache_data = getImpl(cache_base_path, lock);
|
||||
if (cache_data)
|
||||
return cache_data->cache;
|
||||
auto it = caches_by_path.find(cache_base_path);
|
||||
if (it != caches_by_path.end())
|
||||
{
|
||||
caches_by_name.emplace(name, it->second);
|
||||
return it->second->cache;
|
||||
}
|
||||
|
||||
auto cache = std::make_shared<LRUFileCache>(cache_base_path, file_cache_settings);
|
||||
caches.emplace(cache_base_path, CacheData(cache, file_cache_settings));
|
||||
FileCacheData result{cache, file_cache_settings};
|
||||
|
||||
auto cache_it = caches.insert(caches.end(), std::move(result));
|
||||
caches_by_name.emplace(name, cache_it);
|
||||
caches_by_path.emplace(cache_base_path, cache_it);
|
||||
|
||||
return cache;
|
||||
}
|
||||
|
||||
FileCacheFactory::FileCacheData FileCacheFactory::getByName(const std::string & name)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
auto it = caches_by_name.find(name);
|
||||
if (it == caches_by_name.end())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by name: {}", name);
|
||||
|
||||
return *it->second;
|
||||
}
|
||||
|
||||
FileCacheFactory::CacheByName FileCacheFactory::getAllByName()
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return caches_by_name;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <unordered_map>
|
||||
#include <mutex>
|
||||
#include <list>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -15,20 +16,22 @@ namespace DB
|
||||
*/
|
||||
class FileCacheFactory final : private boost::noncopyable
|
||||
{
|
||||
struct CacheData
|
||||
public:
|
||||
struct FileCacheData
|
||||
{
|
||||
FileCachePtr cache;
|
||||
FileCacheSettings settings;
|
||||
|
||||
CacheData(FileCachePtr cache_, const FileCacheSettings & settings_) : cache(cache_), settings(settings_) {}
|
||||
FileCacheData(FileCachePtr cache_, const FileCacheSettings & settings_) : cache(cache_), settings(settings_) {}
|
||||
};
|
||||
|
||||
using CacheByBasePath = std::unordered_map<std::string, CacheData>;
|
||||
using Caches = std::list<FileCacheData>;
|
||||
using CacheByBasePath = std::unordered_map<std::string, Caches::iterator>;
|
||||
using CacheByName = std::unordered_map<std::string, Caches::iterator>;
|
||||
|
||||
public:
|
||||
static FileCacheFactory & instance();
|
||||
|
||||
FileCachePtr getOrCreate(const std::string & cache_base_path, const FileCacheSettings & file_cache_settings);
|
||||
FileCachePtr getOrCreate(const std::string & cache_base_path, const FileCacheSettings & file_cache_settings, const std::string & name);
|
||||
|
||||
FileCachePtr get(const std::string & cache_base_path);
|
||||
|
||||
@ -36,11 +39,16 @@ public:
|
||||
|
||||
const FileCacheSettings & getSettings(const std::string & cache_base_path);
|
||||
|
||||
private:
|
||||
CacheData * getImpl(const std::string & cache_base_path, std::lock_guard<std::mutex> &);
|
||||
FileCacheData getByName(const std::string & name);
|
||||
|
||||
CacheByName getAllByName();
|
||||
|
||||
private:
|
||||
std::mutex mutex;
|
||||
CacheByBasePath caches;
|
||||
Caches caches;
|
||||
|
||||
CacheByBasePath caches_by_path;
|
||||
CacheByName caches_by_name;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -148,28 +148,22 @@ ThreadStatus::ThreadStatus()
|
||||
|
||||
ThreadStatus::~ThreadStatus()
|
||||
{
|
||||
try
|
||||
{
|
||||
if (untracked_memory > 0)
|
||||
memory_tracker.alloc(untracked_memory);
|
||||
else
|
||||
memory_tracker.free(-untracked_memory);
|
||||
}
|
||||
catch (const DB::Exception &)
|
||||
{
|
||||
/// It's a minor tracked memory leak here (not the memory itself but it's counter).
|
||||
/// We've already allocated a little bit more than the limit and cannot track it in the thread memory tracker or its parent.
|
||||
tryLogCurrentException(log);
|
||||
}
|
||||
if (untracked_memory > 0)
|
||||
memory_tracker.allocNoThrow(untracked_memory);
|
||||
else
|
||||
memory_tracker.free(-untracked_memory);
|
||||
|
||||
if (thread_group)
|
||||
{
|
||||
std::lock_guard guard(thread_group->mutex);
|
||||
thread_group->finished_threads_counters_memory.emplace_back(ThreadGroupStatus::ProfileEventsCountersAndMemory{
|
||||
ThreadGroupStatus::ProfileEventsCountersAndMemory counters
|
||||
{
|
||||
performance_counters.getPartiallyAtomicSnapshot(),
|
||||
memory_tracker.get(),
|
||||
thread_id,
|
||||
});
|
||||
thread_id
|
||||
};
|
||||
|
||||
std::lock_guard guard(thread_group->mutex);
|
||||
thread_group->finished_threads_counters_memory.emplace_back(std::move(counters));
|
||||
thread_group->threads.erase(this);
|
||||
}
|
||||
|
||||
|
@ -14,6 +14,7 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Storages/ExternalDataSourceConfiguration.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/Macros.h>
|
||||
|
||||
#include "config_core.h"
|
||||
@ -322,12 +323,25 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
||||
configuration.username = safeGetLiteralValue<String>(engine_args[2], engine_name);
|
||||
configuration.password = safeGetLiteralValue<String>(engine_args[3], engine_name);
|
||||
|
||||
bool is_deprecated_syntax = false;
|
||||
if (engine_args.size() >= 5)
|
||||
configuration.schema = safeGetLiteralValue<String>(engine_args[4], engine_name);
|
||||
}
|
||||
{
|
||||
auto arg_value = engine_args[4]->as<ASTLiteral>()->value;
|
||||
if (arg_value.getType() == Field::Types::Which::String)
|
||||
{
|
||||
configuration.schema = safeGetLiteralValue<String>(engine_args[4], engine_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
use_table_cache = safeGetLiteralValue<UInt8>(engine_args[4], engine_name);
|
||||
LOG_WARNING(&Poco::Logger::get("DatabaseFactory"), "A deprecated syntax of PostgreSQL database engine is used");
|
||||
is_deprecated_syntax = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (engine_args.size() >= 6)
|
||||
use_table_cache = safeGetLiteralValue<UInt8>(engine_args[5], engine_name);
|
||||
if (!is_deprecated_syntax && engine_args.size() >= 6)
|
||||
use_table_cache = safeGetLiteralValue<UInt8>(engine_args[5], engine_name);
|
||||
}
|
||||
|
||||
auto pool = std::make_shared<postgres::PoolWithFailover>(configuration,
|
||||
context->getSettingsRef().postgresql_connection_pool_size,
|
||||
|
@ -263,10 +263,11 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure(
|
||||
"WHERE attrelid = (SELECT oid FROM pg_class WHERE {}) "
|
||||
"AND NOT attisdropped AND attnum > 0", where);
|
||||
|
||||
table.physical_columns = readNamesAndTypesList(tx, postgres_table, query, use_nulls, false);
|
||||
auto postgres_table_with_schema = postgres_schema.empty() ? postgres_table : doubleQuoteString(postgres_schema) + '.' + doubleQuoteString(postgres_table);
|
||||
table.physical_columns = readNamesAndTypesList(tx, postgres_table_with_schema, query, use_nulls, false);
|
||||
|
||||
if (!table.physical_columns)
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "PostgreSQL table {} does not exist", postgres_table);
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "PostgreSQL table {} does not exist", postgres_table_with_schema);
|
||||
|
||||
if (with_primary_key)
|
||||
{
|
||||
@ -278,7 +279,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure(
|
||||
"AND a.attnum = ANY(i.indkey) "
|
||||
"WHERE attrelid = (SELECT oid FROM pg_class WHERE {}) AND i.indisprimary", where);
|
||||
|
||||
table.primary_key_columns = readNamesAndTypesList(tx, postgres_table, query, use_nulls, true);
|
||||
table.primary_key_columns = readNamesAndTypesList(tx, postgres_table_with_schema, query, use_nulls, true);
|
||||
}
|
||||
|
||||
if (with_replica_identity_index && !table.primary_key_columns)
|
||||
@ -299,11 +300,13 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure(
|
||||
"and a.attnum = ANY(ix.indkey) "
|
||||
"and t.relkind in ('r', 'p') " /// simple tables
|
||||
"and t.relname = {} " /// Connection is already done to a needed database, only table name is needed.
|
||||
"{}"
|
||||
"and ix.indisreplident = 't' " /// index is is replica identity index
|
||||
"ORDER BY a.attname", /// column names
|
||||
quoteString(postgres_table));
|
||||
"ORDER BY a.attname", /// column name
|
||||
(postgres_schema.empty() ? "" : "and t.relnamespace = " + quoteString(postgres_schema)) + " ",
|
||||
quoteString(postgres_table));
|
||||
|
||||
table.replica_identity_columns = readNamesAndTypesList(tx, postgres_table, query, use_nulls, true);
|
||||
table.replica_identity_columns = readNamesAndTypesList(tx, postgres_table_with_schema, query, use_nulls, true);
|
||||
}
|
||||
|
||||
return table;
|
||||
|
@ -75,7 +75,7 @@ public:
|
||||
void startup(ContextPtr context) override;
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap & map) override;
|
||||
String getCacheBasePath() const override { return delegate->getCacheBasePath(); }
|
||||
std::vector<String> getRemotePaths(const String & path) const override { return delegate->getRemotePaths(path); }
|
||||
PathsWithSize getObjectStoragePaths(const String & path) const override { return delegate->getObjectStoragePaths(path); }
|
||||
void getRemotePathsRecursive(const String & path, std::vector<LocalPathWithRemotePaths> & paths_map) override { return delegate->getRemotePathsRecursive(path, paths_map); }
|
||||
|
||||
MetadataStoragePtr getMetadataStorage() override { return delegate->getMetadataStorage(); }
|
||||
|
@ -318,10 +318,10 @@ String DiskRestartProxy::getCacheBasePath() const
|
||||
return DiskDecorator::getCacheBasePath();
|
||||
}
|
||||
|
||||
std::vector<String> DiskRestartProxy::getRemotePaths(const String & path) const
|
||||
PathsWithSize DiskRestartProxy::getObjectStoragePaths(const String & path) const
|
||||
{
|
||||
ReadLock lock (mutex);
|
||||
return DiskDecorator::getRemotePaths(path);
|
||||
return DiskDecorator::getObjectStoragePaths(path);
|
||||
}
|
||||
|
||||
void DiskRestartProxy::getRemotePathsRecursive(const String & path, std::vector<LocalPathWithRemotePaths> & paths_map)
|
||||
|
@ -65,7 +65,7 @@ public:
|
||||
String getUniqueId(const String & path) const override;
|
||||
bool checkUniqueId(const String & id) const override;
|
||||
String getCacheBasePath() const override;
|
||||
std::vector<String> getRemotePaths(const String & path) const override;
|
||||
PathsWithSize getObjectStoragePaths(const String & path) const override;
|
||||
void getRemotePathsRecursive(const String & path, std::vector<LocalPathWithRemotePaths> & paths_map) override;
|
||||
|
||||
void restart(ContextPtr context);
|
||||
|
@ -170,10 +170,10 @@ std::unique_ptr<ReadBufferFromFileBase> DiskWebServer::readFile(const String & p
|
||||
auto remote_path = fs_path.parent_path() / (escapeForFileName(fs_path.stem()) + fs_path.extension().string());
|
||||
remote_path = remote_path.string().substr(url.size());
|
||||
|
||||
std::vector<BlobPathWithSize> blobs_to_read;
|
||||
PathsWithSize blobs_to_read;
|
||||
blobs_to_read.emplace_back(remote_path, iter->second.size);
|
||||
|
||||
auto web_impl = std::make_unique<ReadBufferFromWebServerGather>(url, path, blobs_to_read, getContext(), read_settings);
|
||||
auto web_impl = std::make_unique<ReadBufferFromWebServerGather>(url, blobs_to_read, getContext(), read_settings);
|
||||
|
||||
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
|
||||
{
|
||||
|
@ -169,7 +169,7 @@ public:
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Disk {} is read-only", getName());
|
||||
}
|
||||
|
||||
std::vector<String> getRemotePaths(const String &) const override { return {}; }
|
||||
PathsWithSize getObjectStoragePaths(const String &) const override { return {}; }
|
||||
|
||||
void getRemotePathsRecursive(const String &, std::vector<LocalPathWithRemotePaths> &) override {}
|
||||
|
||||
|
@ -219,13 +219,13 @@ public:
|
||||
|
||||
/// Returns a list of paths because for Log family engines there might be
|
||||
/// multiple files in remote fs for single clickhouse file.
|
||||
virtual std::vector<String> getRemotePaths(const String &) const
|
||||
virtual PathsWithSize getObjectStoragePaths(const String &) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getRemotePaths() not implemented for disk: {}`", getType());
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getObjectStoragePaths() not implemented for disk: {}`", getType());
|
||||
}
|
||||
|
||||
/// For one local path there might be multiple remote paths in case of Log family engines.
|
||||
using LocalPathWithRemotePaths = std::pair<String, std::vector<String>>;
|
||||
using LocalPathWithRemotePaths = std::pair<String, PathsWithSize>;
|
||||
|
||||
virtual void getRemotePathsRecursive(const String &, std::vector<LocalPathWithRemotePaths> &)
|
||||
{
|
||||
|
@ -40,7 +40,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c
|
||||
appendFilesystemCacheLog();
|
||||
}
|
||||
|
||||
current_file_path = fs::path(common_path_prefix) / path;
|
||||
current_file_path = path;
|
||||
current_file_size = file_size;
|
||||
total_bytes_read_from_current_file = 0;
|
||||
|
||||
@ -50,18 +50,30 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c
|
||||
#if USE_AWS_S3
|
||||
SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBufferImpl(const String & path, size_t file_size)
|
||||
{
|
||||
auto remote_path = fs::path(common_path_prefix) / path;
|
||||
auto remote_file_reader_creator = [=, this]()
|
||||
{
|
||||
return std::make_unique<ReadBufferFromS3>(
|
||||
client_ptr, bucket, remote_path, version_id, max_single_read_retries,
|
||||
settings, /* use_external_buffer */true, /* offset */ 0, read_until_position, /* restricted_seek */true);
|
||||
client_ptr,
|
||||
bucket,
|
||||
path,
|
||||
version_id,
|
||||
max_single_read_retries,
|
||||
settings,
|
||||
/* use_external_buffer */true,
|
||||
/* offset */0,
|
||||
read_until_position,
|
||||
/* restricted_seek */true);
|
||||
};
|
||||
|
||||
if (with_cache)
|
||||
{
|
||||
return std::make_shared<CachedReadBufferFromRemoteFS>(
|
||||
remote_path, settings.remote_fs_cache, remote_file_reader_creator, settings, query_id, read_until_position ? read_until_position : file_size);
|
||||
path,
|
||||
settings.remote_fs_cache,
|
||||
remote_file_reader_creator,
|
||||
settings,
|
||||
query_id,
|
||||
read_until_position ? read_until_position : file_size);
|
||||
}
|
||||
|
||||
return remote_file_reader_creator();
|
||||
@ -72,34 +84,46 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBufferImpl(con
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
SeekableReadBufferPtr ReadBufferFromAzureBlobStorageGather::createImplementationBufferImpl(const String & path, size_t /* file_size */)
|
||||
{
|
||||
current_file_path = path;
|
||||
return std::make_unique<ReadBufferFromAzureBlobStorage>(blob_container_client, path, max_single_read_retries,
|
||||
max_single_download_retries, settings.remote_fs_buffer_size, /* use_external_buffer */true, read_until_position);
|
||||
return std::make_unique<ReadBufferFromAzureBlobStorage>(
|
||||
blob_container_client,
|
||||
path,
|
||||
max_single_read_retries,
|
||||
max_single_download_retries,
|
||||
settings.remote_fs_buffer_size,
|
||||
/* use_external_buffer */true,
|
||||
read_until_position);
|
||||
}
|
||||
#endif
|
||||
|
||||
|
||||
SeekableReadBufferPtr ReadBufferFromWebServerGather::createImplementationBufferImpl(const String & path, size_t /* file_size */)
|
||||
{
|
||||
current_file_path = path;
|
||||
return std::make_unique<ReadBufferFromWebServer>(fs::path(uri) / path, context, settings, /* use_external_buffer */true, read_until_position);
|
||||
return std::make_unique<ReadBufferFromWebServer>(
|
||||
fs::path(uri) / path,
|
||||
context,
|
||||
settings,
|
||||
/* use_external_buffer */true,
|
||||
read_until_position);
|
||||
}
|
||||
|
||||
|
||||
#if USE_HDFS
|
||||
SeekableReadBufferPtr ReadBufferFromHDFSGather::createImplementationBufferImpl(const String & path, size_t /* file_size */)
|
||||
{
|
||||
return std::make_unique<ReadBufferFromHDFS>(hdfs_uri, fs::path(hdfs_directory) / path, config, settings.remote_fs_buffer_size);
|
||||
size_t begin_of_path = path.find('/', path.find("//") + 2);
|
||||
auto hdfs_path = path.substr(begin_of_path);
|
||||
auto hdfs_uri = path.substr(0, begin_of_path);
|
||||
LOG_TEST(log, "HDFS uri: {}, path: {}", hdfs_path, hdfs_uri);
|
||||
|
||||
return std::make_unique<ReadBufferFromHDFS>(hdfs_uri, hdfs_path, config);
|
||||
}
|
||||
#endif
|
||||
|
||||
|
||||
ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(
|
||||
const std::string & common_path_prefix_,
|
||||
const BlobsPathToSize & blobs_to_read_,
|
||||
const PathsWithSize & blobs_to_read_,
|
||||
const ReadSettings & settings_)
|
||||
: ReadBuffer(nullptr, 0)
|
||||
, common_path_prefix(common_path_prefix_)
|
||||
, blobs_to_read(blobs_to_read_)
|
||||
, settings(settings_)
|
||||
, query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "")
|
||||
|
@ -27,8 +27,7 @@ friend class ReadIndirectBufferFromRemoteFS;
|
||||
|
||||
public:
|
||||
ReadBufferFromRemoteFSGather(
|
||||
const std::string & common_path_prefix_,
|
||||
const BlobsPathToSize & blobs_to_read_,
|
||||
const PathsWithSize & blobs_to_read_,
|
||||
const ReadSettings & settings_);
|
||||
|
||||
~ReadBufferFromRemoteFSGather() override;
|
||||
@ -54,9 +53,7 @@ public:
|
||||
protected:
|
||||
virtual SeekableReadBufferPtr createImplementationBufferImpl(const String & path, size_t file_size) = 0;
|
||||
|
||||
std::string common_path_prefix;
|
||||
|
||||
BlobsPathToSize blobs_to_read;
|
||||
PathsWithSize blobs_to_read;
|
||||
|
||||
ReadSettings settings;
|
||||
|
||||
@ -69,6 +66,8 @@ protected:
|
||||
|
||||
String query_id;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
||||
private:
|
||||
SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t file_size);
|
||||
|
||||
@ -95,8 +94,6 @@ private:
|
||||
*/
|
||||
size_t bytes_to_ignore = 0;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
||||
size_t total_bytes_read_from_current_file = 0;
|
||||
|
||||
bool enable_cache_log = false;
|
||||
@ -112,11 +109,10 @@ public:
|
||||
std::shared_ptr<const Aws::S3::S3Client> client_ptr_,
|
||||
const String & bucket_,
|
||||
const String & version_id_,
|
||||
const std::string & common_path_prefix_,
|
||||
const BlobsPathToSize & blobs_to_read_,
|
||||
const PathsWithSize & blobs_to_read_,
|
||||
size_t max_single_read_retries_,
|
||||
const ReadSettings & settings_)
|
||||
: ReadBufferFromRemoteFSGather(common_path_prefix_, blobs_to_read_, settings_)
|
||||
: ReadBufferFromRemoteFSGather(blobs_to_read_, settings_)
|
||||
, client_ptr(std::move(client_ptr_))
|
||||
, bucket(bucket_)
|
||||
, version_id(version_id_)
|
||||
@ -142,12 +138,11 @@ class ReadBufferFromAzureBlobStorageGather final : public ReadBufferFromRemoteFS
|
||||
public:
|
||||
ReadBufferFromAzureBlobStorageGather(
|
||||
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
const std::string & common_path_prefix_,
|
||||
const BlobsPathToSize & blobs_to_read_,
|
||||
const PathsWithSize & blobs_to_read_,
|
||||
size_t max_single_read_retries_,
|
||||
size_t max_single_download_retries_,
|
||||
const ReadSettings & settings_)
|
||||
: ReadBufferFromRemoteFSGather(common_path_prefix_, blobs_to_read_, settings_)
|
||||
: ReadBufferFromRemoteFSGather(blobs_to_read_, settings_)
|
||||
, blob_container_client(blob_container_client_)
|
||||
, max_single_read_retries(max_single_read_retries_)
|
||||
, max_single_download_retries(max_single_download_retries_)
|
||||
@ -169,11 +164,10 @@ class ReadBufferFromWebServerGather final : public ReadBufferFromRemoteFSGather
|
||||
public:
|
||||
ReadBufferFromWebServerGather(
|
||||
const String & uri_,
|
||||
const std::string & common_path_prefix_,
|
||||
const BlobsPathToSize & blobs_to_read_,
|
||||
const PathsWithSize & blobs_to_read_,
|
||||
ContextPtr context_,
|
||||
const ReadSettings & settings_)
|
||||
: ReadBufferFromRemoteFSGather(common_path_prefix_, blobs_to_read_, settings_)
|
||||
: ReadBufferFromRemoteFSGather(blobs_to_read_, settings_)
|
||||
, uri(uri_)
|
||||
, context(context_)
|
||||
{
|
||||
@ -194,25 +188,19 @@ class ReadBufferFromHDFSGather final : public ReadBufferFromRemoteFSGather
|
||||
public:
|
||||
ReadBufferFromHDFSGather(
|
||||
const Poco::Util::AbstractConfiguration & config_,
|
||||
const String & hdfs_uri_,
|
||||
const std::string & common_path_prefix_,
|
||||
const BlobsPathToSize & blobs_to_read_,
|
||||
const PathsWithSize & blobs_to_read_,
|
||||
const ReadSettings & settings_)
|
||||
: ReadBufferFromRemoteFSGather(common_path_prefix_, blobs_to_read_, settings_)
|
||||
: ReadBufferFromRemoteFSGather(blobs_to_read_, settings_)
|
||||
, config(config_)
|
||||
{
|
||||
const size_t begin_of_path = hdfs_uri_.find('/', hdfs_uri_.find("//") + 2);
|
||||
hdfs_directory = hdfs_uri_.substr(begin_of_path);
|
||||
hdfs_uri = hdfs_uri_.substr(0, begin_of_path);
|
||||
}
|
||||
|
||||
SeekableReadBufferPtr createImplementationBufferImpl(const String & path, size_t file_size) override;
|
||||
|
||||
private:
|
||||
const Poco::Util::AbstractConfiguration & config;
|
||||
String hdfs_uri;
|
||||
String hdfs_directory;
|
||||
};
|
||||
|
||||
#endif
|
||||
|
||||
}
|
||||
|
@ -19,7 +19,8 @@ class ReadBufferFromWebServer : public SeekableReadBuffer
|
||||
{
|
||||
public:
|
||||
explicit ReadBufferFromWebServer(
|
||||
const String & url_, ContextPtr context_,
|
||||
const String & url_,
|
||||
ContextPtr context_,
|
||||
const ReadSettings & settings_ = {},
|
||||
bool use_external_buffer_ = false,
|
||||
size_t read_until_position = 0);
|
||||
|
@ -67,16 +67,18 @@ std::unique_ptr<SeekableReadBuffer> AzureObjectStorage::readObject( /// NOLINT
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> AzureObjectStorage::readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const PathsWithSize & paths_to_read,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
auto settings_ptr = settings.get();
|
||||
auto reader_impl = std::make_unique<ReadBufferFromAzureBlobStorageGather>(
|
||||
client.get(), common_path_prefix, blobs_to_read,
|
||||
settings_ptr->max_single_read_retries, settings_ptr->max_single_download_retries, read_settings);
|
||||
client.get(),
|
||||
paths_to_read,
|
||||
settings_ptr->max_single_read_retries,
|
||||
settings_ptr->max_single_download_retries,
|
||||
read_settings);
|
||||
|
||||
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
|
||||
{
|
||||
@ -111,7 +113,7 @@ std::unique_ptr<WriteBufferFromFileBase> AzureObjectStorage::writeObject( /// NO
|
||||
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(buffer), std::move(finalize_callback), path);
|
||||
}
|
||||
|
||||
void AzureObjectStorage::listPrefix(const std::string & path, BlobsPathToSize & children) const
|
||||
void AzureObjectStorage::listPrefix(const std::string & path, PathsWithSize & children) const
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
|
||||
@ -134,10 +136,10 @@ void AzureObjectStorage::removeObject(const std::string & path)
|
||||
throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Failed to delete file in AzureBlob Storage: {}", path);
|
||||
}
|
||||
|
||||
void AzureObjectStorage::removeObjects(const std::vector<std::string> & paths)
|
||||
void AzureObjectStorage::removeObjects(const PathsWithSize & paths)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
for (const auto & path : paths)
|
||||
for (const auto & [path, _] : paths)
|
||||
{
|
||||
auto delete_info = client_ptr->DeleteBlob(path);
|
||||
if (!delete_info.Value.Deleted)
|
||||
@ -151,10 +153,10 @@ void AzureObjectStorage::removeObjectIfExists(const std::string & path)
|
||||
auto delete_info = client_ptr->DeleteBlob(path);
|
||||
}
|
||||
|
||||
void AzureObjectStorage::removeObjectsIfExist(const std::vector<std::string> & paths)
|
||||
void AzureObjectStorage::removeObjectsIfExist(const PathsWithSize & paths)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
for (const auto & path : paths)
|
||||
for (const auto & [path, _] : paths)
|
||||
auto delete_info = client_ptr->DeleteBlob(path);
|
||||
}
|
||||
|
||||
|
@ -59,8 +59,7 @@ public:
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const PathsWithSize & blobs_to_read,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
@ -74,15 +73,16 @@ public:
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
const WriteSettings & write_settings = {}) override;
|
||||
|
||||
void listPrefix(const std::string & path, BlobsPathToSize & children) const override;
|
||||
void listPrefix(const std::string & path, PathsWithSize & children) const override;
|
||||
|
||||
/// Remove file. Throws exception if file doesn't exists or it's a directory.
|
||||
void removeObject(const std::string & path) override;
|
||||
|
||||
void removeObjects(const std::vector<std::string> & paths) override;
|
||||
void removeObjects(const PathsWithSize & paths) override;
|
||||
|
||||
void removeObjectIfExists(const std::string & path) override;
|
||||
|
||||
void removeObjectsIfExist(const std::vector<std::string> & paths) override;
|
||||
void removeObjectsIfExist(const PathsWithSize & paths) override;
|
||||
|
||||
ObjectMetadata getObjectMetadata(const std::string & path) const override;
|
||||
|
||||
@ -95,11 +95,19 @@ public:
|
||||
|
||||
void startup() override {}
|
||||
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) override;
|
||||
void applyNewSettings(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
ContextPtr context) override;
|
||||
|
||||
String getObjectsNamespace() const override { return ""; }
|
||||
|
||||
std::unique_ptr<IObjectStorage> cloneObjectStorage(const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) override;
|
||||
std::unique_ptr<IObjectStorage> cloneObjectStorage(
|
||||
const std::string & new_namespace,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
ContextPtr context) override;
|
||||
|
||||
|
||||
private:
|
||||
const String name;
|
||||
|
@ -108,9 +108,9 @@ DiskObjectStorage::DiskObjectStorage(
|
||||
, metadata_helper(std::make_unique<DiskObjectStorageRemoteMetadataRestoreHelper>(this, ReadSettings{}))
|
||||
{}
|
||||
|
||||
std::vector<String> DiskObjectStorage::getRemotePaths(const String & local_path) const
|
||||
PathsWithSize DiskObjectStorage::getObjectStoragePaths(const String & local_path) const
|
||||
{
|
||||
return metadata_storage->getRemotePaths(local_path);
|
||||
return metadata_storage->getObjectStoragePaths(local_path);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std::vector<LocalPathWithRemotePaths> & paths_map)
|
||||
@ -120,7 +120,7 @@ void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std::
|
||||
{
|
||||
try
|
||||
{
|
||||
paths_map.emplace_back(local_path, getRemotePaths(local_path));
|
||||
paths_map.emplace_back(local_path, getObjectStoragePaths(local_path));
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
@ -244,9 +244,9 @@ String DiskObjectStorage::getUniqueId(const String & path) const
|
||||
{
|
||||
LOG_TRACE(log, "Remote path: {}, Path: {}", remote_fs_root_path, path);
|
||||
String id;
|
||||
auto blobs_paths = metadata_storage->getRemotePaths(path);
|
||||
auto blobs_paths = metadata_storage->getObjectStoragePaths(path);
|
||||
if (!blobs_paths.empty())
|
||||
id = blobs_paths[0];
|
||||
id = blobs_paths[0].path;
|
||||
return id;
|
||||
}
|
||||
|
||||
@ -438,7 +438,11 @@ std::unique_ptr<ReadBufferFromFileBase> DiskObjectStorage::readFile(
|
||||
std::optional<size_t> read_hint,
|
||||
std::optional<size_t> file_size) const
|
||||
{
|
||||
return object_storage->readObjects(remote_fs_root_path, metadata_storage->getBlobs(path), settings, read_hint, file_size);
|
||||
return object_storage->readObjects(
|
||||
metadata_storage->getObjectStoragePaths(path),
|
||||
settings,
|
||||
read_hint,
|
||||
file_size);
|
||||
}
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> DiskObjectStorage::writeFile(
|
||||
|
@ -49,7 +49,7 @@ public:
|
||||
|
||||
const String & getPath() const override { return metadata_storage->getPath(); }
|
||||
|
||||
std::vector<String> getRemotePaths(const String & local_path) const override;
|
||||
PathsWithSize getObjectStoragePaths(const String & local_path) const override;
|
||||
|
||||
void getRemotePathsRecursive(const String & local_path, std::vector<LocalPathWithRemotePaths> & paths_map) override;
|
||||
|
||||
|
@ -73,7 +73,7 @@ FileCachePtr getCachePtrForDisk(
|
||||
FileCacheSettings file_cache_settings;
|
||||
file_cache_settings.loadFromConfig(config, config_prefix);
|
||||
|
||||
auto cache = FileCacheFactory::instance().getOrCreate(cache_base_path, file_cache_settings);
|
||||
auto cache = FileCacheFactory::instance().getOrCreate(cache_base_path, file_cache_settings, name);
|
||||
cache->initialize();
|
||||
|
||||
auto * log = &Poco::Logger::get("Disk(" + name + ")");
|
||||
|
@ -11,6 +11,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_FORMAT;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf)
|
||||
@ -26,14 +27,14 @@ void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf)
|
||||
|
||||
assertChar('\n', buf);
|
||||
|
||||
UInt32 remote_fs_objects_count;
|
||||
readIntText(remote_fs_objects_count, buf);
|
||||
UInt32 storage_objects_count;
|
||||
readIntText(storage_objects_count, buf);
|
||||
assertChar('\t', buf);
|
||||
readIntText(total_size, buf);
|
||||
assertChar('\n', buf);
|
||||
remote_fs_objects.resize(remote_fs_objects_count);
|
||||
storage_objects.resize(storage_objects_count);
|
||||
|
||||
for (size_t i = 0; i < remote_fs_objects_count; ++i)
|
||||
for (size_t i = 0; i < storage_objects_count; ++i)
|
||||
{
|
||||
String remote_fs_object_path;
|
||||
size_t remote_fs_object_size;
|
||||
@ -50,8 +51,8 @@ void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf)
|
||||
remote_fs_object_path = remote_fs_object_path.substr(remote_fs_root_path.size());
|
||||
}
|
||||
assertChar('\n', buf);
|
||||
remote_fs_objects[i].relative_path = remote_fs_object_path;
|
||||
remote_fs_objects[i].bytes_size = remote_fs_object_size;
|
||||
storage_objects[i].relative_path = remote_fs_object_path;
|
||||
storage_objects[i].bytes_size = remote_fs_object_size;
|
||||
}
|
||||
|
||||
readIntText(ref_count, buf);
|
||||
@ -75,12 +76,12 @@ void DiskObjectStorageMetadata::serialize(WriteBuffer & buf, bool sync) const
|
||||
writeIntText(VERSION_READ_ONLY_FLAG, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
writeIntText(remote_fs_objects.size(), buf);
|
||||
writeIntText(storage_objects.size(), buf);
|
||||
writeChar('\t', buf);
|
||||
writeIntText(total_size, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
for (const auto & [remote_fs_object_path, remote_fs_object_size] : remote_fs_objects)
|
||||
for (const auto & [remote_fs_object_path, remote_fs_object_size] : storage_objects)
|
||||
{
|
||||
writeIntText(remote_fs_object_size, buf);
|
||||
writeChar('\t', buf);
|
||||
@ -119,8 +120,11 @@ DiskObjectStorageMetadata::DiskObjectStorageMetadata(
|
||||
|
||||
void DiskObjectStorageMetadata::addObject(const String & path, size_t size)
|
||||
{
|
||||
if (!remote_fs_root_path.empty() && path.starts_with(remote_fs_root_path))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected relative path");
|
||||
|
||||
total_size += size;
|
||||
remote_fs_objects.emplace_back(path, size);
|
||||
storage_objects.emplace_back(path, size);
|
||||
}
|
||||
|
||||
|
||||
|
@ -12,6 +12,17 @@ namespace DB
|
||||
struct DiskObjectStorageMetadata
|
||||
{
|
||||
private:
|
||||
struct RelativePathWithSize
|
||||
{
|
||||
String relative_path;
|
||||
size_t bytes_size;
|
||||
|
||||
RelativePathWithSize() = default;
|
||||
|
||||
RelativePathWithSize(const String & relative_path_, size_t bytes_size_)
|
||||
: relative_path(relative_path_), bytes_size(bytes_size_) {}
|
||||
};
|
||||
|
||||
/// Metadata file version.
|
||||
static constexpr uint32_t VERSION_ABSOLUTE_PATHS = 1;
|
||||
static constexpr uint32_t VERSION_RELATIVE_PATHS = 2;
|
||||
@ -19,8 +30,8 @@ private:
|
||||
|
||||
const std::string & common_metadata_path;
|
||||
|
||||
/// Remote FS objects paths and their sizes.
|
||||
std::vector<BlobPathWithSize> remote_fs_objects;
|
||||
/// Relative paths of blobs.
|
||||
std::vector<RelativePathWithSize> storage_objects;
|
||||
|
||||
/// URI
|
||||
const std::string & remote_fs_root_path;
|
||||
@ -60,9 +71,9 @@ public:
|
||||
return remote_fs_root_path;
|
||||
}
|
||||
|
||||
std::vector<BlobPathWithSize> getBlobs() const
|
||||
std::vector<RelativePathWithSize> getBlobsRelativePaths() const
|
||||
{
|
||||
return remote_fs_objects;
|
||||
return storage_objects;
|
||||
}
|
||||
|
||||
bool isReadOnly() const
|
||||
|
@ -84,13 +84,13 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::migrateFileToRestorableSchema
|
||||
{
|
||||
LOG_TRACE(disk->log, "Migrate file {} to restorable schema", disk->metadata_storage->getPath() + path);
|
||||
|
||||
auto blobs = disk->metadata_storage->getBlobs(path);
|
||||
for (const auto & [key, _] : blobs)
|
||||
auto objects = disk->metadata_storage->getObjectStoragePaths(path);
|
||||
for (const auto & [object_path, _] : objects)
|
||||
{
|
||||
ObjectAttributes metadata {
|
||||
{"path", path}
|
||||
};
|
||||
updateObjectMetadata(disk->remote_fs_root_path + key, metadata);
|
||||
updateObjectMetadata(object_path, metadata);
|
||||
}
|
||||
}
|
||||
void DiskObjectStorageRemoteMetadataRestoreHelper::migrateToRestorableSchemaRecursive(const String & path, Futures & results)
|
||||
@ -346,7 +346,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFiles(IObjectStorage *
|
||||
LOG_INFO(disk->log, "Starting restore files for disk {}", disk->name);
|
||||
|
||||
std::vector<std::future<void>> results;
|
||||
auto restore_files = [this, &source_object_storage, &restore_information, &results](const BlobsPathToSize & keys)
|
||||
auto restore_files = [this, &source_object_storage, &restore_information, &results](const PathsWithSize & keys)
|
||||
{
|
||||
std::vector<String> keys_names;
|
||||
for (const auto & [key, size] : keys)
|
||||
@ -379,7 +379,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFiles(IObjectStorage *
|
||||
return true;
|
||||
};
|
||||
|
||||
BlobsPathToSize children;
|
||||
PathsWithSize children;
|
||||
source_object_storage->listPrefix(restore_information.source_path, children);
|
||||
|
||||
restore_files(children);
|
||||
@ -456,7 +456,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject
|
||||
bool send_metadata = source_object_storage->getObjectsNamespace() != disk->object_storage->getObjectsNamespace() || disk->remote_fs_root_path != restore_information.source_path;
|
||||
|
||||
std::set<String> renames;
|
||||
auto restore_file_operations = [this, &source_object_storage, &restore_information, &renames, &send_metadata](const BlobsPathToSize & keys)
|
||||
auto restore_file_operations = [this, &source_object_storage, &restore_information, &renames, &send_metadata](const PathsWithSize & keys)
|
||||
{
|
||||
const String rename = "rename";
|
||||
const String hardlink = "hardlink";
|
||||
@ -523,7 +523,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject
|
||||
return true;
|
||||
};
|
||||
|
||||
BlobsPathToSize children;
|
||||
PathsWithSize children;
|
||||
source_object_storage->listPrefix(restore_information.source_path + "operations/", children);
|
||||
restore_file_operations(children);
|
||||
|
||||
|
@ -65,7 +65,7 @@ struct RemoveObjectStorageOperation final : public IDiskObjectStorageOperation
|
||||
std::string path;
|
||||
bool delete_metadata_only;
|
||||
bool remove_from_cache{false};
|
||||
std::vector<std::string> paths_to_remove;
|
||||
PathsWithSize paths_to_remove;
|
||||
bool if_exists;
|
||||
|
||||
RemoveObjectStorageOperation(
|
||||
@ -96,13 +96,13 @@ struct RemoveObjectStorageOperation final : public IDiskObjectStorageOperation
|
||||
try
|
||||
{
|
||||
uint32_t hardlink_count = metadata_storage.getHardlinkCount(path);
|
||||
auto remote_objects = metadata_storage.getRemotePaths(path);
|
||||
auto objects = metadata_storage.getObjectStoragePaths(path);
|
||||
|
||||
tx->unlinkMetadata(path);
|
||||
|
||||
if (hardlink_count == 0)
|
||||
{
|
||||
paths_to_remove = remote_objects;
|
||||
paths_to_remove = objects;
|
||||
remove_from_cache = true;
|
||||
}
|
||||
}
|
||||
@ -134,7 +134,7 @@ struct RemoveObjectStorageOperation final : public IDiskObjectStorageOperation
|
||||
if (remove_from_cache)
|
||||
{
|
||||
for (const auto & path_to_remove : paths_to_remove)
|
||||
object_storage.removeFromCache(path_to_remove);
|
||||
object_storage.removeFromCache(path_to_remove.path);
|
||||
}
|
||||
|
||||
}
|
||||
@ -143,10 +143,10 @@ struct RemoveObjectStorageOperation final : public IDiskObjectStorageOperation
|
||||
struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOperation
|
||||
{
|
||||
std::string path;
|
||||
std::unordered_map<std::string, std::vector<std::string>> paths_to_remove;
|
||||
std::unordered_map<std::string, PathsWithSize> paths_to_remove;
|
||||
bool keep_all_batch_data;
|
||||
NameSet file_names_remove_metadata_only;
|
||||
std::vector<std::string> path_to_remove_from_cache;
|
||||
PathsWithSize path_to_remove_from_cache;
|
||||
|
||||
RemoveRecursiveObjectStorageOperation(
|
||||
IObjectStorage & object_storage_,
|
||||
@ -169,14 +169,14 @@ struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOp
|
||||
try
|
||||
{
|
||||
uint32_t hardlink_count = metadata_storage.getHardlinkCount(path_to_remove);
|
||||
auto remote_objects = metadata_storage.getRemotePaths(path_to_remove);
|
||||
auto objects_paths = metadata_storage.getObjectStoragePaths(path_to_remove);
|
||||
|
||||
tx->unlinkMetadata(path_to_remove);
|
||||
|
||||
if (hardlink_count == 0)
|
||||
{
|
||||
paths_to_remove[path_to_remove] = remote_objects;
|
||||
path_to_remove_from_cache.insert(path_to_remove_from_cache.end(), remote_objects.begin(), remote_objects.end());
|
||||
paths_to_remove[path_to_remove] = objects_paths;
|
||||
path_to_remove_from_cache.insert(path_to_remove_from_cache.end(), objects_paths.begin(), objects_paths.end());
|
||||
}
|
||||
|
||||
}
|
||||
@ -217,7 +217,7 @@ struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOp
|
||||
{
|
||||
if (!keep_all_batch_data)
|
||||
{
|
||||
std::vector<std::string> remove_from_remote;
|
||||
PathsWithSize remove_from_remote;
|
||||
for (auto && [local_path, remote_paths] : paths_to_remove)
|
||||
{
|
||||
if (!file_names_remove_metadata_only.contains(fs::path(local_path).filename()))
|
||||
@ -228,7 +228,7 @@ struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOp
|
||||
object_storage.removeObjects(remove_from_remote);
|
||||
}
|
||||
|
||||
for (const auto & path_to_remove : path_to_remove_from_cache)
|
||||
for (const auto & [path_to_remove, _] : path_to_remove_from_cache)
|
||||
object_storage.removeFromCache(path_to_remove);
|
||||
}
|
||||
};
|
||||
@ -238,7 +238,7 @@ struct ReplaceFileObjectStorageOperation final : public IDiskObjectStorageOperat
|
||||
{
|
||||
std::string path_from;
|
||||
std::string path_to;
|
||||
std::vector<std::string> blobs_to_remove;
|
||||
PathsWithSize blobs_to_remove;
|
||||
|
||||
ReplaceFileObjectStorageOperation(
|
||||
IObjectStorage & object_storage_,
|
||||
@ -254,7 +254,7 @@ struct ReplaceFileObjectStorageOperation final : public IDiskObjectStorageOperat
|
||||
{
|
||||
if (metadata_storage.exists(path_to))
|
||||
{
|
||||
blobs_to_remove = metadata_storage.getRemotePaths(path_to);
|
||||
blobs_to_remove = metadata_storage.getObjectStoragePaths(path_to);
|
||||
tx->replaceFile(path_from, path_to);
|
||||
}
|
||||
else
|
||||
@ -328,14 +328,15 @@ struct CopyFileObjectStorageOperation final : public IDiskObjectStorageOperation
|
||||
void execute(MetadataTransactionPtr tx) override
|
||||
{
|
||||
tx->createEmptyMetadataFile(to_path);
|
||||
auto source_blobs = metadata_storage.getBlobs(from_path);
|
||||
auto source_blobs = metadata_storage.getObjectStoragePaths(from_path); /// Full paths
|
||||
|
||||
for (const auto & [blob_from, size] : source_blobs)
|
||||
{
|
||||
auto blob_name = getRandomASCIIString();
|
||||
|
||||
auto blob_to = fs::path(remote_fs_root_path) / blob_name;
|
||||
|
||||
object_storage.copyObject(fs::path(remote_fs_root_path) / blob_from, blob_to);
|
||||
object_storage.copyObject(blob_from, blob_to);
|
||||
|
||||
tx->addBlobToMetadata(to_path, blob_name, size);
|
||||
|
||||
|
@ -49,13 +49,12 @@ std::unique_ptr<SeekableReadBuffer> HDFSObjectStorage::readObject( /// NOLINT
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> HDFSObjectStorage::readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const PathsWithSize & paths_to_read,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
auto hdfs_impl = std::make_unique<ReadBufferFromHDFSGather>(config, common_path_prefix, common_path_prefix, blobs_to_read, read_settings);
|
||||
auto hdfs_impl = std::make_unique<ReadBufferFromHDFSGather>(config, paths_to_read, read_settings);
|
||||
auto buf = std::make_unique<ReadIndirectBufferFromRemoteFS>(std::move(hdfs_impl));
|
||||
return std::make_unique<SeekAvoidingReadBuffer>(std::move(buf), settings->min_bytes_for_seek);
|
||||
}
|
||||
@ -69,7 +68,9 @@ std::unique_ptr<WriteBufferFromFileBase> HDFSObjectStorage::writeObject( /// NOL
|
||||
const WriteSettings &)
|
||||
{
|
||||
if (attributes.has_value())
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "HDFS API doesn't support custom attributes/metadata for stored objects");
|
||||
throw Exception(
|
||||
ErrorCodes::UNSUPPORTED_METHOD,
|
||||
"HDFS API doesn't support custom attributes/metadata for stored objects");
|
||||
|
||||
/// Single O_WRONLY in libhdfs adds O_TRUNC
|
||||
auto hdfs_buffer = std::make_unique<WriteBufferFromHDFS>(
|
||||
@ -80,7 +81,7 @@ std::unique_ptr<WriteBufferFromFileBase> HDFSObjectStorage::writeObject( /// NOL
|
||||
}
|
||||
|
||||
|
||||
void HDFSObjectStorage::listPrefix(const std::string & path, BlobsPathToSize & children) const
|
||||
void HDFSObjectStorage::listPrefix(const std::string & path, PathsWithSize & children) const
|
||||
{
|
||||
const size_t begin_of_path = path.find('/', path.find("//") + 2);
|
||||
int32_t num_entries;
|
||||
@ -104,10 +105,10 @@ void HDFSObjectStorage::removeObject(const std::string & path)
|
||||
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::removeObjects(const std::vector<std::string> & paths)
|
||||
void HDFSObjectStorage::removeObjects(const PathsWithSize & paths)
|
||||
{
|
||||
for (const auto & hdfs_path : paths)
|
||||
removeObject(hdfs_path);
|
||||
for (const auto & [path, _] : paths)
|
||||
removeObject(path);
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::removeObjectIfExists(const std::string & path)
|
||||
@ -116,15 +117,17 @@ void HDFSObjectStorage::removeObjectIfExists(const std::string & path)
|
||||
removeObject(path);
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::removeObjectsIfExist(const std::vector<std::string> & paths)
|
||||
void HDFSObjectStorage::removeObjectsIfExist(const PathsWithSize & paths)
|
||||
{
|
||||
for (const auto & hdfs_path : paths)
|
||||
removeObjectIfExists(hdfs_path);
|
||||
for (const auto & [path, _] : paths)
|
||||
removeObjectIfExists(path);
|
||||
}
|
||||
|
||||
ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string &) const
|
||||
{
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "HDFS API doesn't support custom attributes/metadata for stored objects");
|
||||
throw Exception(
|
||||
ErrorCodes::UNSUPPORTED_METHOD,
|
||||
"HDFS API doesn't support custom attributes/metadata for stored objects");
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::copyObject( /// NOLINT
|
||||
@ -133,7 +136,9 @@ void HDFSObjectStorage::copyObject( /// NOLINT
|
||||
std::optional<ObjectAttributes> object_to_attributes)
|
||||
{
|
||||
if (object_to_attributes.has_value())
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "HDFS API doesn't support custom attributes/metadata for stored objects");
|
||||
throw Exception(
|
||||
ErrorCodes::UNSUPPORTED_METHOD,
|
||||
"HDFS API doesn't support custom attributes/metadata for stored objects");
|
||||
|
||||
auto in = readObject(object_from);
|
||||
auto out = writeObject(object_to, WriteMode::Rewrite);
|
||||
|
@ -61,8 +61,7 @@ public:
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const PathsWithSize & paths_to_read,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
@ -76,15 +75,16 @@ public:
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
const WriteSettings & write_settings = {}) override;
|
||||
|
||||
void listPrefix(const std::string & path, BlobsPathToSize & children) const override;
|
||||
void listPrefix(const std::string & path, PathsWithSize & children) const override;
|
||||
|
||||
/// Remove file. Throws exception if file doesn't exists or it's a directory.
|
||||
void removeObject(const std::string & path) override;
|
||||
|
||||
void removeObjects(const std::vector<std::string> & paths) override;
|
||||
void removeObjects(const PathsWithSize & paths) override;
|
||||
|
||||
void removeObjectIfExists(const std::string & path) override;
|
||||
|
||||
void removeObjectsIfExist(const std::vector<std::string> & paths) override;
|
||||
void removeObjectsIfExist(const PathsWithSize & paths) override;
|
||||
|
||||
ObjectMetadata getObjectMetadata(const std::string & path) const override;
|
||||
|
||||
@ -97,11 +97,18 @@ public:
|
||||
|
||||
void startup() override;
|
||||
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) override;
|
||||
void applyNewSettings(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
ContextPtr context) override;
|
||||
|
||||
String getObjectsNamespace() const override { return ""; }
|
||||
|
||||
std::unique_ptr<IObjectStorage> cloneObjectStorage(const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) override;
|
||||
std::unique_ptr<IObjectStorage> cloneObjectStorage(
|
||||
const std::string & new_namespace,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
ContextPtr context) override;
|
||||
|
||||
private:
|
||||
const Poco::Util::AbstractConfiguration & config;
|
||||
@ -110,8 +117,6 @@ private:
|
||||
HDFSFSPtr hdfs_fs;
|
||||
|
||||
SettingsPtr settings;
|
||||
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -119,11 +119,9 @@ public:
|
||||
/// Read multiple metadata files into strings and return mapping from file_path -> metadata
|
||||
virtual std::unordered_map<std::string, std::string> getSerializedMetadata(const std::vector<String> & file_paths) const = 0;
|
||||
|
||||
/// Return list of paths corresponding to metadata stored in local path
|
||||
virtual std::vector<std::string> getRemotePaths(const std::string & path) const = 0;
|
||||
|
||||
/// Return [(remote_path, size_in_bytes), ...] for metadata path
|
||||
virtual BlobsPathToSize getBlobs(const std::string & path) const = 0;
|
||||
/// Return [(object_storage_path, size_in_bytes), ...] for metadata path
|
||||
/// object_storage_path is a full path to the blob.
|
||||
virtual PathsWithSize getObjectStoragePaths(const std::string & path) const = 0;
|
||||
};
|
||||
|
||||
using MetadataStoragePtr = std::shared_ptr<IMetadataStorage>;
|
||||
|
@ -25,23 +25,23 @@ class WriteBufferFromFileBase;
|
||||
|
||||
using ObjectAttributes = std::map<std::string, std::string>;
|
||||
|
||||
/// Path to blob with it's size
|
||||
struct BlobPathWithSize
|
||||
/// Path to a file and its size.
|
||||
/// Path can be either relative or absolute - according to the context of use.
|
||||
struct PathWithSize
|
||||
{
|
||||
std::string relative_path;
|
||||
std::string path;
|
||||
uint64_t bytes_size;
|
||||
|
||||
BlobPathWithSize() = default;
|
||||
BlobPathWithSize(const BlobPathWithSize & other) = default;
|
||||
PathWithSize() = default;
|
||||
|
||||
BlobPathWithSize(const std::string & relative_path_, uint64_t bytes_size_)
|
||||
: relative_path(relative_path_)
|
||||
PathWithSize(const std::string & path_, uint64_t bytes_size_)
|
||||
: path(path_)
|
||||
, bytes_size(bytes_size_)
|
||||
{}
|
||||
};
|
||||
|
||||
/// List of blobs with their sizes
|
||||
using BlobsPathToSize = std::vector<BlobPathWithSize>;
|
||||
/// List of paths with their sizes
|
||||
using PathsWithSize = std::vector<PathWithSize>;
|
||||
|
||||
struct ObjectMetadata
|
||||
{
|
||||
@ -65,8 +65,8 @@ public:
|
||||
/// Path exists or not
|
||||
virtual bool exists(const std::string & path) const = 0;
|
||||
|
||||
/// List on prefix, return children with their sizes.
|
||||
virtual void listPrefix(const std::string & path, BlobsPathToSize & children) const = 0;
|
||||
/// List on prefix, return children (relative paths) with their sizes.
|
||||
virtual void listPrefix(const std::string & path, PathsWithSize & children) const = 0;
|
||||
|
||||
/// Get object metadata if supported. It should be possible to receive
|
||||
/// at least size of object
|
||||
@ -81,8 +81,7 @@ public:
|
||||
|
||||
/// Read multiple objects with common prefix
|
||||
virtual std::unique_ptr<ReadBufferFromFileBase> readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const PathsWithSize & paths_to_read,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const = 0;
|
||||
@ -101,13 +100,13 @@ public:
|
||||
|
||||
/// Remove multiple objects. Some object storages can do batch remove in a more
|
||||
/// optimal way.
|
||||
virtual void removeObjects(const std::vector<std::string> & paths) = 0;
|
||||
virtual void removeObjects(const PathsWithSize & paths) = 0;
|
||||
|
||||
/// Remove object on path if exists
|
||||
virtual void removeObjectIfExists(const std::string & path) = 0;
|
||||
|
||||
/// Remove objects on path if exists
|
||||
virtual void removeObjectsIfExist(const std::vector<std::string> & paths) = 0;
|
||||
virtual void removeObjectsIfExist(const PathsWithSize & paths) = 0;
|
||||
|
||||
/// Copy object with different attributes if required
|
||||
virtual void copyObject( /// NOLINT
|
||||
@ -140,7 +139,10 @@ public:
|
||||
void removeFromCache(const std::string & path);
|
||||
|
||||
/// Apply new settings, in most cases reiniatilize client and some other staff
|
||||
virtual void applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) = 0;
|
||||
virtual void applyNewSettings(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
ContextPtr context) = 0;
|
||||
|
||||
/// Sometimes object storages have something similar to chroot or namespace, for example
|
||||
/// buckets in S3. If object storage doesn't have any namepaces return empty string.
|
||||
@ -148,7 +150,10 @@ public:
|
||||
|
||||
/// FIXME: confusing function required for a very specific case. Create new instance of object storage
|
||||
/// in different namespace.
|
||||
virtual std::unique_ptr<IObjectStorage> cloneObjectStorage(const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) = 0;
|
||||
virtual std::unique_ptr<IObjectStorage> cloneObjectStorage(
|
||||
const std::string & new_namespace,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix, ContextPtr context) = 0;
|
||||
|
||||
protected:
|
||||
FileCachePtr cache;
|
||||
|
@ -300,18 +300,21 @@ MetadataTransactionPtr MetadataStorageFromDisk::createTransaction() const
|
||||
return std::make_shared<MetadataStorageFromDiskTransaction>(*this);
|
||||
}
|
||||
|
||||
std::vector<std::string> MetadataStorageFromDisk::getRemotePaths(const std::string & path) const
|
||||
PathsWithSize MetadataStorageFromDisk::getObjectStoragePaths(const std::string & path) const
|
||||
{
|
||||
auto metadata = readMetadata(path);
|
||||
|
||||
std::vector<std::string> remote_paths;
|
||||
auto blobs = metadata->getBlobs();
|
||||
auto root_path = metadata->getBlobsCommonPrefix();
|
||||
remote_paths.reserve(blobs.size());
|
||||
for (const auto & [remote_path, _] : blobs)
|
||||
remote_paths.push_back(fs::path(root_path) / remote_path);
|
||||
auto object_storage_relative_paths = metadata->getBlobsRelativePaths(); /// Relative paths.
|
||||
fs::path root_path = metadata->getBlobsCommonPrefix();
|
||||
|
||||
return remote_paths;
|
||||
PathsWithSize object_storage_paths;
|
||||
object_storage_paths.reserve(object_storage_relative_paths.size());
|
||||
|
||||
/// Relative paths -> absolute.
|
||||
for (auto & [object_relative_path, size] : object_storage_relative_paths)
|
||||
object_storage_paths.emplace_back(root_path / object_relative_path, size);
|
||||
|
||||
return object_storage_paths;
|
||||
}
|
||||
|
||||
uint32_t MetadataStorageFromDisk::getHardlinkCount(const std::string & path) const
|
||||
@ -320,12 +323,6 @@ uint32_t MetadataStorageFromDisk::getHardlinkCount(const std::string & path) con
|
||||
return metadata->getRefCount();
|
||||
}
|
||||
|
||||
BlobsPathToSize MetadataStorageFromDisk::getBlobs(const std::string & path) const
|
||||
{
|
||||
auto metadata = readMetadata(path);
|
||||
return metadata->getBlobs();
|
||||
}
|
||||
|
||||
void MetadataStorageFromDiskTransaction::unlinkMetadata(const std::string & path)
|
||||
{
|
||||
auto metadata = metadata_storage.readMetadata(path);
|
||||
|
@ -59,9 +59,7 @@ public:
|
||||
|
||||
std::unordered_map<String, String> getSerializedMetadata(const std::vector<String> & file_paths) const override;
|
||||
|
||||
BlobsPathToSize getBlobs(const std::string & path) const override;
|
||||
|
||||
std::vector<std::string> getRemotePaths(const std::string & path) const override;
|
||||
PathsWithSize getObjectStoragePaths(const std::string & path) const override;
|
||||
|
||||
uint32_t getHardlinkCount(const std::string & path) const override;
|
||||
|
||||
|
@ -109,8 +109,7 @@ bool S3ObjectStorage::exists(const std::string & path) const
|
||||
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> S3ObjectStorage::readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const PathsWithSize & paths_to_read,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
@ -128,8 +127,12 @@ std::unique_ptr<ReadBufferFromFileBase> S3ObjectStorage::readObjects( /// NOLINT
|
||||
auto settings_ptr = s3_settings.get();
|
||||
|
||||
auto s3_impl = std::make_unique<ReadBufferFromS3Gather>(
|
||||
client.get(), bucket, version_id, common_path_prefix, blobs_to_read,
|
||||
settings_ptr->s3_settings.max_single_read_retries, disk_read_settings);
|
||||
client.get(),
|
||||
bucket,
|
||||
version_id,
|
||||
paths_to_read,
|
||||
settings_ptr->s3_settings.max_single_read_retries,
|
||||
disk_read_settings);
|
||||
|
||||
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
|
||||
{
|
||||
@ -192,7 +195,7 @@ std::unique_ptr<WriteBufferFromFileBase> S3ObjectStorage::writeObject( /// NOLIN
|
||||
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(s3_buffer), std::move(finalize_callback), path);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::listPrefix(const std::string & path, BlobsPathToSize & children) const
|
||||
void S3ObjectStorage::listPrefix(const std::string & path, PathsWithSize & children) const
|
||||
{
|
||||
auto settings_ptr = s3_settings.get();
|
||||
auto client_ptr = client.get();
|
||||
@ -253,14 +256,14 @@ void S3ObjectStorage::removeObjectImpl(const std::string & path, bool if_exists)
|
||||
}
|
||||
}
|
||||
|
||||
void S3ObjectStorage::removeObjectsImpl(const std::vector<std::string> & paths, bool if_exists)
|
||||
void S3ObjectStorage::removeObjectsImpl(const PathsWithSize & paths, bool if_exists)
|
||||
{
|
||||
if (paths.empty())
|
||||
return;
|
||||
|
||||
if (!s3_capabilities.support_batch_delete)
|
||||
{
|
||||
for (const auto & path : paths)
|
||||
for (const auto & [path, _] : paths)
|
||||
removeObjectImpl(path, if_exists);
|
||||
}
|
||||
else
|
||||
@ -278,12 +281,12 @@ void S3ObjectStorage::removeObjectsImpl(const std::vector<std::string> & paths,
|
||||
for (; current_position < paths.size() && current_chunk.size() < chunk_size_limit; ++current_position)
|
||||
{
|
||||
Aws::S3::Model::ObjectIdentifier obj;
|
||||
obj.SetKey(paths[current_position]);
|
||||
obj.SetKey(paths[current_position].path);
|
||||
current_chunk.push_back(obj);
|
||||
|
||||
if (!keys.empty())
|
||||
keys += ", ";
|
||||
keys += paths[current_position];
|
||||
keys += paths[current_position].path;
|
||||
}
|
||||
|
||||
Aws::S3::Model::Delete delkeys;
|
||||
@ -308,12 +311,12 @@ void S3ObjectStorage::removeObjectIfExists(const std::string & path)
|
||||
removeObjectImpl(path, true);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::removeObjects(const std::vector<std::string> & paths)
|
||||
void S3ObjectStorage::removeObjects(const PathsWithSize & paths)
|
||||
{
|
||||
removeObjectsImpl(paths, false);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::removeObjectsIfExist(const std::vector<std::string> & paths)
|
||||
void S3ObjectStorage::removeObjectsIfExist(const PathsWithSize & paths)
|
||||
{
|
||||
removeObjectsImpl(paths, true);
|
||||
}
|
||||
|
@ -66,8 +66,7 @@ public:
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const PathsWithSize & paths_to_read,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
@ -81,15 +80,16 @@ public:
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
const WriteSettings & write_settings = {}) override;
|
||||
|
||||
void listPrefix(const std::string & path, BlobsPathToSize & children) const override;
|
||||
void listPrefix(const std::string & path, PathsWithSize & children) const override;
|
||||
|
||||
/// Remove file. Throws exception if file doesn't exist or it's a directory.
|
||||
void removeObject(const std::string & path) override;
|
||||
|
||||
void removeObjects(const std::vector<std::string> & paths) override;
|
||||
void removeObjects(const PathsWithSize & paths) override;
|
||||
|
||||
void removeObjectIfExists(const std::string & path) override;
|
||||
|
||||
void removeObjectsIfExist(const std::vector<std::string> & paths) override;
|
||||
void removeObjectsIfExist(const PathsWithSize & paths) override;
|
||||
|
||||
ObjectMetadata getObjectMetadata(const std::string & path) const override;
|
||||
|
||||
@ -108,26 +108,42 @@ public:
|
||||
|
||||
void startup() override;
|
||||
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) override;
|
||||
void applyNewSettings(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
ContextPtr context) override;
|
||||
|
||||
String getObjectsNamespace() const override { return bucket; }
|
||||
|
||||
std::unique_ptr<IObjectStorage> cloneObjectStorage(const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) override;
|
||||
std::unique_ptr<IObjectStorage> cloneObjectStorage(
|
||||
const std::string & new_namespace,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
ContextPtr context) override;
|
||||
|
||||
private:
|
||||
void setNewSettings(std::unique_ptr<S3ObjectStorageSettings> && s3_settings_);
|
||||
|
||||
void setNewClient(std::unique_ptr<Aws::S3::S3Client> && client_);
|
||||
|
||||
void copyObjectImpl(const String & src_bucket, const String & src_key, const String & dst_bucket, const String & dst_key,
|
||||
void copyObjectImpl(
|
||||
const String & src_bucket,
|
||||
const String & src_key,
|
||||
const String & dst_bucket,
|
||||
const String & dst_key,
|
||||
std::optional<Aws::S3::Model::HeadObjectResult> head = std::nullopt,
|
||||
std::optional<ObjectAttributes> metadata = std::nullopt) const;
|
||||
|
||||
void copyObjectMultipartImpl(const String & src_bucket, const String & src_key, const String & dst_bucket, const String & dst_key,
|
||||
void copyObjectMultipartImpl(
|
||||
const String & src_bucket,
|
||||
const String & src_key,
|
||||
const String & dst_bucket,
|
||||
const String & dst_key,
|
||||
std::optional<Aws::S3::Model::HeadObjectResult> head = std::nullopt,
|
||||
std::optional<ObjectAttributes> metadata = std::nullopt) const;
|
||||
|
||||
void removeObjectImpl(const std::string & path, bool if_exists);
|
||||
void removeObjectsImpl(const std::vector<std::string> & paths, bool if_exists);
|
||||
void removeObjectsImpl(const PathsWithSize & paths, bool if_exists);
|
||||
|
||||
Aws::S3::Model::HeadObjectOutcome requestObjectHeadData(const std::string & bucket_from, const std::string & key) const;
|
||||
|
||||
|
@ -72,6 +72,10 @@ if (TARGET ch_contrib::llvm)
|
||||
target_link_libraries(clickhouse_functions PRIVATE ch_contrib::llvm)
|
||||
endif ()
|
||||
|
||||
if (TARGET ch_contrib::base-x)
|
||||
target_link_libraries(clickhouse_functions PRIVATE ch_contrib::base-x)
|
||||
endif()
|
||||
|
||||
if (TARGET ch_contrib::base64)
|
||||
target_link_libraries(clickhouse_functions PRIVATE ch_contrib::base64)
|
||||
endif()
|
||||
@ -86,8 +90,8 @@ if (TARGET ch_contrib::h3)
|
||||
target_link_libraries (clickhouse_functions PRIVATE ch_contrib::h3)
|
||||
endif()
|
||||
|
||||
if (TARGET ch_contrib::hyperscan)
|
||||
target_link_libraries(clickhouse_functions PRIVATE ch_contrib::hyperscan)
|
||||
if (TARGET ch_contrib::vectorscan)
|
||||
target_link_libraries(clickhouse_functions PRIVATE ch_contrib::vectorscan)
|
||||
endif()
|
||||
|
||||
if (TARGET ch_contrib::simdjson)
|
||||
|
256
src/Functions/FunctionBase58Conversion.h
Normal file
256
src/Functions/FunctionBase58Conversion.h
Normal file
@ -0,0 +1,256 @@
|
||||
#pragma once
|
||||
#include "config_functions.h"
|
||||
|
||||
#if USE_BASEX
|
||||
# include <Columns/ColumnConst.h>
|
||||
# include <Common/MemorySanitizer.h>
|
||||
# include <Columns/ColumnString.h>
|
||||
# include <DataTypes/DataTypeString.h>
|
||||
# include <Functions/FunctionFactory.h>
|
||||
# include <Functions/FunctionHelpers.h>
|
||||
# include <IO/WriteHelpers.h>
|
||||
# include <base_x.hh>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
struct Base58Encode
|
||||
{
|
||||
static constexpr auto name = "base58Encode";
|
||||
|
||||
static void process(const ColumnString & input, ColumnString::MutablePtr & dst_column, const std::string & alphabet, size_t input_rows_count)
|
||||
{
|
||||
auto & dst_data = dst_column->getChars();
|
||||
auto & dst_offsets = dst_column->getOffsets();
|
||||
|
||||
/// Wikipedia states Base58 has efficiency of 73%, and we take 1.5 scale to avoid reallocation in most cases
|
||||
size_t current_allocated_size = ceil(1.5 * input.getChars().size());
|
||||
|
||||
dst_data.resize(current_allocated_size);
|
||||
dst_offsets.resize(input_rows_count);
|
||||
|
||||
const ColumnString::Offsets & src_offsets = input.getOffsets();
|
||||
|
||||
const auto * source = input.getChars().raw_data();
|
||||
auto * dst = dst_data.data();
|
||||
auto * dst_pos = dst;
|
||||
|
||||
size_t src_offset_prev = 0;
|
||||
size_t processed_size = 0;
|
||||
|
||||
const auto& encoder = (alphabet == "bitcoin") ? Base58::bitcoin() :
|
||||
((alphabet == "flickr") ? Base58::flickr() :
|
||||
((alphabet == "ripple") ? Base58::ripple() :
|
||||
Base58::base58())); //GMP
|
||||
|
||||
std::string encoded;
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
{
|
||||
size_t srclen = src_offsets[row] - src_offset_prev - 1;
|
||||
/// Why we didn't use char* here?
|
||||
/// We don't know the size of the result string beforehand (it's not byte-to-byte encoding),
|
||||
/// so we may need to do many resizes (the worst case -- we'll do it for each row)
|
||||
/// This way we do exponential resizes and one final resize after whole operation is complete
|
||||
encoded.clear();
|
||||
if (srclen)
|
||||
try
|
||||
{
|
||||
encoder.encode(encoded, source, srclen);
|
||||
}
|
||||
catch (const std::invalid_argument& e)
|
||||
{
|
||||
throw Exception(e.what(), ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
catch (const std::domain_error& e)
|
||||
{
|
||||
throw Exception(e.what(), ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
size_t outlen = encoded.size();
|
||||
|
||||
if (processed_size + outlen >= current_allocated_size)
|
||||
{
|
||||
current_allocated_size += current_allocated_size;
|
||||
dst_data.resize(current_allocated_size);
|
||||
auto processed_offset = dst_pos - dst;
|
||||
dst = dst_data.data();
|
||||
dst_pos = dst;
|
||||
dst_pos += processed_offset;
|
||||
}
|
||||
std::memcpy(dst_pos, encoded.c_str(), ++outlen);
|
||||
|
||||
source += srclen + 1;
|
||||
dst_pos += outlen;
|
||||
|
||||
dst_offsets[row] = dst_pos - dst;
|
||||
src_offset_prev = src_offsets[row];
|
||||
processed_size += outlen;
|
||||
}
|
||||
|
||||
dst_data.resize(dst_pos - dst);
|
||||
}
|
||||
};
|
||||
|
||||
struct Base58Decode
|
||||
{
|
||||
static constexpr auto name = "base58Decode";
|
||||
|
||||
static void process(const ColumnString & input, ColumnString::MutablePtr & dst_column, const std::string & alphabet, size_t input_rows_count)
|
||||
{
|
||||
auto & dst_data = dst_column->getChars();
|
||||
auto & dst_offsets = dst_column->getOffsets();
|
||||
|
||||
/// We allocate probably even more then needed to avoid many resizes
|
||||
size_t current_allocated_size = input.getChars().size();
|
||||
|
||||
dst_data.resize(current_allocated_size);
|
||||
dst_offsets.resize(input_rows_count);
|
||||
|
||||
const ColumnString::Offsets & src_offsets = input.getOffsets();
|
||||
|
||||
const auto * source = input.getChars().raw_data();
|
||||
auto * dst = dst_data.data();
|
||||
auto * dst_pos = dst;
|
||||
|
||||
size_t src_offset_prev = 0;
|
||||
size_t processed_size = 0;
|
||||
|
||||
const auto& decoder = (alphabet == "bitcoin") ? Base58::bitcoin() :
|
||||
((alphabet == "flickr") ? Base58::flickr() :
|
||||
((alphabet == "ripple") ? Base58::ripple() :
|
||||
Base58::base58()));
|
||||
|
||||
std::string decoded;
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
{
|
||||
size_t srclen = src_offsets[row] - src_offset_prev - 1;
|
||||
/// Why we didn't use char* here?
|
||||
/// We don't know the size of the result string beforehand (it's not byte-to-byte encoding),
|
||||
/// so we may need to do many resizes (the worst case -- we'll do it for each row)
|
||||
/// This way we do exponential resizes and one final resize after whole operation is complete
|
||||
decoded.clear();
|
||||
if (srclen)
|
||||
try
|
||||
{
|
||||
decoder.decode(decoded, source, srclen);
|
||||
}
|
||||
catch (const std::invalid_argument& e)
|
||||
{
|
||||
throw Exception(e.what(), ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
catch (const std::domain_error& e)
|
||||
{
|
||||
throw Exception(e.what(), ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
size_t outlen = decoded.size();
|
||||
|
||||
if (processed_size + outlen >= current_allocated_size)
|
||||
{
|
||||
current_allocated_size += current_allocated_size;
|
||||
dst_data.resize(current_allocated_size);
|
||||
auto processed_offset = dst_pos - dst;
|
||||
dst = dst_data.data();
|
||||
dst_pos = dst;
|
||||
dst_pos += processed_offset;
|
||||
}
|
||||
std::memcpy(dst_pos, decoded.c_str(), ++outlen);
|
||||
|
||||
source += srclen + 1;
|
||||
dst_pos += outlen;
|
||||
|
||||
dst_offsets[row] = dst_pos - dst;
|
||||
src_offset_prev = src_offsets[row];
|
||||
processed_size += outlen;
|
||||
}
|
||||
|
||||
dst_data.resize(dst_pos - dst);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Func>
|
||||
class FunctionBase58Conversion : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Func::name;
|
||||
|
||||
static FunctionPtr create(ContextPtr)
|
||||
{
|
||||
return std::make_shared<FunctionBase58Conversion>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return Func::name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 1 && arguments.size() != 2)
|
||||
throw Exception(
|
||||
"Wrong number of arguments for function " + getName() + ": 1 or 2 expected.",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (!isString(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of 1st argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() == 2 && !isString(arguments[1].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1].type->getName() + " of 2nd argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
const ColumnPtr column_string = arguments[0].column;
|
||||
const ColumnString * input = checkAndGetColumn<ColumnString>(column_string.get());
|
||||
if (!input)
|
||||
throw Exception(
|
||||
"Illegal column " + arguments[0].column->getName() + " of first argument of function " + getName() + ", must be String",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
std::string alphabet = "bitcoin";
|
||||
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const auto * alphabet_column = checkAndGetColumn<ColumnConst>(arguments[1].column.get());
|
||||
|
||||
if (!alphabet_column)
|
||||
throw Exception("Second argument for function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
alphabet = alphabet_column->getValue<DB::String>();
|
||||
if (alphabet != "bitcoin" && alphabet != "ripple" && alphabet != "flickr" && alphabet != "gmp")
|
||||
throw Exception("Second argument for function " + getName() + " must be 'bitcoin', 'ripple', 'gmp' or 'flickr'", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
}
|
||||
|
||||
auto dst_column = ColumnString::create();
|
||||
|
||||
Func::process(*input, dst_column, alphabet, input_rows_count);
|
||||
|
||||
return dst_column;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
#endif
|
@ -85,7 +85,7 @@ public:
|
||||
{
|
||||
if (!WhichDataType(arguments[0].type).isString())
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + ". Must be String.",
|
||||
"Illegal type " + arguments[0].type->getName() + " of 1st argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
@ -98,7 +98,7 @@ public:
|
||||
|
||||
if (!input)
|
||||
throw Exception(
|
||||
"Illegal column " + arguments[0].column->getName() + " of first argument of function " + getName(),
|
||||
"Illegal column " + arguments[0].column->getName() + " of first argument of function " + getName() + ", must be of type String",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
auto dst_column = ColumnString::create();
|
||||
|
17
src/Functions/FunctionsBase58.cpp
Normal file
17
src/Functions/FunctionsBase58.cpp
Normal file
@ -0,0 +1,17 @@
|
||||
#include <Functions/FunctionBase58Conversion.h>
|
||||
#if USE_BASEX
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
void registerFunctionBase58Encode(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionBase58Conversion<Base58Encode>>();
|
||||
}
|
||||
|
||||
void registerFunctionBase58Decode(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionBase58Conversion<Base58Decode>>();
|
||||
}
|
||||
}
|
||||
#endif
|
@ -545,6 +545,8 @@ ColumnPtr FunctionAnyArityLogical<Impl, Name>::executeShortCircuit(ColumnsWithTy
|
||||
if (Name::name != NameAnd::name && Name::name != NameOr::name)
|
||||
throw Exception("Function " + getName() + " doesn't support short circuit execution", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
executeColumnIfNeeded(arguments[0]);
|
||||
|
||||
/// Let's denote x_i' = maskedExecute(x_i, mask).
|
||||
/// 1) AND(x_0, x_1, x_2, ..., x_n)
|
||||
/// We will support mask_i = x_0 & x_1 & ... & x_i.
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include "config_functions.h"
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_HYPERSCAN
|
||||
#if USE_VECTORSCAN
|
||||
# include <hs.h>
|
||||
#endif
|
||||
|
||||
@ -60,7 +60,7 @@ struct MultiMatchAllIndicesImpl
|
||||
[[maybe_unused]] std::optional<UInt32> edit_distance)
|
||||
{
|
||||
offsets.resize(haystack_offsets.size());
|
||||
#if USE_HYPERSCAN
|
||||
#if USE_VECTORSCAN
|
||||
const auto & hyperscan_regex = MultiRegexps::get</*SaveIndices=*/true, MultiSearchDistance>(needles, edit_distance);
|
||||
hs_scratch_t * scratch = nullptr;
|
||||
hs_error_t err = hs_clone_scratch(hyperscan_regex->getScratch(), &scratch);
|
||||
@ -97,7 +97,7 @@ struct MultiMatchAllIndicesImpl
|
||||
on_match,
|
||||
&res);
|
||||
if (err != HS_SUCCESS)
|
||||
throw Exception("Failed to scan with hyperscan", ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT);
|
||||
throw Exception("Failed to scan with vectorscan", ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT);
|
||||
offsets[i] = res.size();
|
||||
offset = haystack_offsets[i];
|
||||
}
|
||||
@ -108,9 +108,9 @@ struct MultiMatchAllIndicesImpl
|
||||
(void)res;
|
||||
(void)offsets;
|
||||
throw Exception(
|
||||
"multi-search all indices is not implemented when hyperscan is off (is it x86 processor?)",
|
||||
"multi-search all indices is not implemented when vectorscan is off",
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
#endif // USE_HYPERSCAN
|
||||
#endif // USE_VECTORSCAN
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -8,7 +8,7 @@
|
||||
#include "config_functions.h"
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_HYPERSCAN
|
||||
#if USE_VECTORSCAN
|
||||
# include <hs.h>
|
||||
#else
|
||||
# include "MatchImpl.h"
|
||||
@ -64,13 +64,13 @@ struct MultiMatchAnyImpl
|
||||
(void)FindAny;
|
||||
(void)FindAnyIndex;
|
||||
res.resize(haystack_offsets.size());
|
||||
#if USE_HYPERSCAN
|
||||
#if USE_VECTORSCAN
|
||||
const auto & hyperscan_regex = MultiRegexps::get<FindAnyIndex, MultiSearchDistance>(needles, edit_distance);
|
||||
hs_scratch_t * scratch = nullptr;
|
||||
hs_error_t err = hs_clone_scratch(hyperscan_regex->getScratch(), &scratch);
|
||||
|
||||
if (err != HS_SUCCESS)
|
||||
throw Exception("Could not clone scratch space for hyperscan", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
throw Exception("Could not clone scratch space for vectorscan", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
|
||||
MultiRegexps::ScratchPtr smart_scratch(scratch);
|
||||
|
||||
@ -92,7 +92,7 @@ struct MultiMatchAnyImpl
|
||||
for (size_t i = 0; i < haystack_offsets_size; ++i)
|
||||
{
|
||||
UInt64 length = haystack_offsets[i] - offset - 1;
|
||||
/// Hyperscan restriction.
|
||||
/// Vectorscan restriction.
|
||||
if (length > std::numeric_limits<UInt32>::max())
|
||||
throw Exception("Too long string to search", ErrorCodes::TOO_MANY_BYTES);
|
||||
/// Zero the result, scan, check, update the offset.
|
||||
@ -106,14 +106,14 @@ struct MultiMatchAnyImpl
|
||||
on_match,
|
||||
&res[i]);
|
||||
if (err != HS_SUCCESS && err != HS_SCAN_TERMINATED)
|
||||
throw Exception("Failed to scan with hyperscan", ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT);
|
||||
throw Exception("Failed to scan with vectorscan", ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT);
|
||||
offset = haystack_offsets[i];
|
||||
}
|
||||
#else
|
||||
/// Fallback if do not use hyperscan
|
||||
/// Fallback if do not use vectorscan
|
||||
if constexpr (MultiSearchDistance)
|
||||
throw Exception(
|
||||
"Edit distance multi-search is not implemented when hyperscan is off (is it x86 processor?)",
|
||||
"Edit distance multi-search is not implemented when vectorscan is off",
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
PaddedPODArray<UInt8> accum(res.size());
|
||||
memset(res.data(), 0, res.size() * sizeof(res.front()));
|
||||
@ -129,7 +129,7 @@ struct MultiMatchAnyImpl
|
||||
res[i] = j + 1;
|
||||
}
|
||||
}
|
||||
#endif // USE_HYPERSCAN
|
||||
#endif // USE_VECTORSCAN
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -16,7 +16,7 @@
|
||||
|
||||
#include "config_functions.h"
|
||||
|
||||
#if USE_HYPERSCAN
|
||||
#if USE_VECTORSCAN
|
||||
# include <hs.h>
|
||||
#endif
|
||||
|
||||
@ -103,7 +103,7 @@ private:
|
||||
|
||||
}
|
||||
|
||||
#if USE_HYPERSCAN
|
||||
#if USE_VECTORSCAN
|
||||
|
||||
namespace MultiRegexps
|
||||
{
|
||||
@ -312,6 +312,6 @@ inline Regexps * get(const std::vector<StringRef> & patterns, std::optional<UInt
|
||||
|
||||
}
|
||||
|
||||
#endif // USE_HYPERSCAN
|
||||
#endif // USE_VECTORSCAN
|
||||
|
||||
}
|
||||
|
@ -8,8 +8,8 @@ if (STRIP_DEBUG_SYMBOLS_FUNCTIONS)
|
||||
endif()
|
||||
|
||||
# TODO: move Functions/Regexps.h to some lib and use here
|
||||
if (TARGET ch_contrib::hyperscan)
|
||||
target_link_libraries(clickhouse_functions_url PRIVATE ch_contrib::hyperscan)
|
||||
if (TARGET ch_contrib::vectorscan)
|
||||
target_link_libraries(clickhouse_functions_url PRIVATE ch_contrib::vectorscan)
|
||||
endif()
|
||||
|
||||
if (USE_GPERF)
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
// .h autogenerated by cmake!
|
||||
|
||||
#cmakedefine01 USE_BASEX
|
||||
#cmakedefine01 USE_BASE64
|
||||
#cmakedefine01 USE_SIMDJSON
|
||||
#cmakedefine01 USE_RAPIDJSON
|
||||
@ -9,4 +10,4 @@
|
||||
#cmakedefine01 USE_S2_GEOMETRY
|
||||
#cmakedefine01 USE_FASTOPS
|
||||
#cmakedefine01 USE_NLP
|
||||
#cmakedefine01 USE_HYPERSCAN
|
||||
#cmakedefine01 USE_VECTORSCAN
|
||||
|
@ -1,6 +1,9 @@
|
||||
if (TARGET ch_contrib::fastops)
|
||||
set(USE_FASTOPS 1)
|
||||
endif()
|
||||
if (TARGET ch_contrib::base-x)
|
||||
set(USE_BASEX 1)
|
||||
endif()
|
||||
if (TARGET ch_contrib::base64)
|
||||
set(USE_BASE64 1)
|
||||
endif()
|
||||
@ -16,6 +19,6 @@ endif()
|
||||
if (TARGET ch_contrib::h3)
|
||||
set(USE_H3 1)
|
||||
endif()
|
||||
if (TARGET ch_contrib::hyperscan)
|
||||
set(USE_HYPERSCAN 1)
|
||||
if (TARGET ch_contrib::vectorscan)
|
||||
set(USE_VECTORSCAN 1)
|
||||
endif()
|
||||
|
@ -36,7 +36,7 @@ public:
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
{
|
||||
return std::make_shared<FilesystemImpl<Impl>>(std::filesystem::space(context->getConfigRef().getString("path")));
|
||||
return std::make_shared<FilesystemImpl<Impl>>(std::filesystem::space(context->getPath()));
|
||||
}
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override
|
||||
|
@ -987,6 +987,8 @@ private:
|
||||
if (last_short_circuit_argument_index == -1)
|
||||
return;
|
||||
|
||||
executeColumnIfNeeded(arguments[0]);
|
||||
|
||||
/// Check if condition is const or null to not create full mask from it.
|
||||
if ((isColumnConst(*arguments[0].column) || arguments[0].column->onlyNull()) && !arguments[0].column->empty())
|
||||
{
|
||||
|
@ -266,6 +266,8 @@ private:
|
||||
if (last_short_circuit_argument_index < 0)
|
||||
return;
|
||||
|
||||
executeColumnIfNeeded(arguments[0]);
|
||||
|
||||
/// Let's denote x_i' = maskedExecute(x_i, mask).
|
||||
/// multiIf(x_0, y_0, x_1, y_1, x_2, y_2, ..., x_{n-1}, y_{n-1}, y_n)
|
||||
/// We will support mask_i = !x_0 & !x_1 & ... & !x_i
|
||||
|
@ -49,6 +49,11 @@ void registerFunctionBase64Decode(FunctionFactory &);
|
||||
void registerFunctionTryBase64Decode(FunctionFactory &);
|
||||
#endif
|
||||
|
||||
#if USE_BASEX
|
||||
void registerFunctionBase58Encode(FunctionFactory &);
|
||||
void registerFunctionBase58Decode(FunctionFactory &);
|
||||
#endif
|
||||
|
||||
#if USE_NLP
|
||||
void registerFunctionStem(FunctionFactory &);
|
||||
void registerFunctionSynonyms(FunctionFactory &);
|
||||
@ -105,6 +110,11 @@ void registerFunctionsString(FunctionFactory & factory)
|
||||
registerFunctionTryBase64Decode(factory);
|
||||
#endif
|
||||
|
||||
#if USE_BASEX
|
||||
registerFunctionBase58Encode(factory);
|
||||
registerFunctionBase58Decode(factory);
|
||||
#endif
|
||||
|
||||
#if USE_NLP
|
||||
registerFunctionStem(factory);
|
||||
registerFunctionSynonyms(factory);
|
||||
|
@ -28,14 +28,6 @@ public:
|
||||
|
||||
bool useDefaultImplementationForNothing() const override { return false; }
|
||||
|
||||
bool isShortCircuit(ShortCircuitSettings & settings, size_t /*number_of_arguments*/) const override
|
||||
{
|
||||
settings.enable_lazy_execution_for_first_argument = true;
|
||||
settings.enable_lazy_execution_for_common_descendants_of_arguments = true;
|
||||
settings.force_enable_lazy_execution = true;
|
||||
return true;
|
||||
}
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
|
@ -32,13 +32,6 @@ public:
|
||||
|
||||
bool useDefaultImplementationForNothing() const override { return false; }
|
||||
|
||||
bool isShortCircuit(ShortCircuitSettings & settings, size_t /*number_of_arguments*/) const override
|
||||
{
|
||||
settings.enable_lazy_execution_for_first_argument = false;
|
||||
settings.enable_lazy_execution_for_common_descendants_of_arguments = true;
|
||||
settings.force_enable_lazy_execution = true;
|
||||
return true;
|
||||
}
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
|
||||
|
@ -1786,7 +1786,9 @@ ActionsDAGPtr ActionsDAG::cloneActionsForConjunction(NodeRawConstPtrs conjunctio
|
||||
actions->inputs.push_back(input);
|
||||
}
|
||||
|
||||
actions->index.push_back(input);
|
||||
/// We should not add result_predicate into the index for the second time.
|
||||
if (input->result_name != result_predicate->result_name)
|
||||
actions->index.push_back(input);
|
||||
}
|
||||
|
||||
return actions;
|
||||
@ -1840,13 +1842,14 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown(
|
||||
if (can_remove_filter)
|
||||
{
|
||||
/// If filter column is not needed, remove it from index.
|
||||
for (auto i = index.begin(); i != index.end(); ++i)
|
||||
std::erase_if(index, [&](const Node * node) { return node == predicate; });
|
||||
|
||||
/// At the very end of this method we'll call removeUnusedActions() with allow_remove_inputs=false,
|
||||
/// so we need to manually remove predicate if it is an input node.
|
||||
if (predicate->type == ActionType::INPUT)
|
||||
{
|
||||
if (*i == predicate)
|
||||
{
|
||||
index.erase(i);
|
||||
break;
|
||||
}
|
||||
std::erase_if(inputs, [&](const Node * node) { return node == predicate; });
|
||||
nodes.remove_if([&](const Node & node) { return &node == predicate; });
|
||||
}
|
||||
}
|
||||
else
|
||||
|
@ -82,24 +82,6 @@ void AggregateDescription::explain(WriteBuffer & out, size_t indent) const
|
||||
}
|
||||
out << "\n";
|
||||
}
|
||||
|
||||
out << prefix << " Argument positions: ";
|
||||
|
||||
if (arguments.empty())
|
||||
out << "none\n";
|
||||
else
|
||||
{
|
||||
bool first = true;
|
||||
for (auto arg : arguments)
|
||||
{
|
||||
if (!first)
|
||||
out << ", ";
|
||||
first = false;
|
||||
|
||||
out << arg;
|
||||
}
|
||||
out << '\n';
|
||||
}
|
||||
}
|
||||
|
||||
void AggregateDescription::explain(JSONBuilder::JSONMap & map) const
|
||||
@ -137,15 +119,6 @@ void AggregateDescription::explain(JSONBuilder::JSONMap & map) const
|
||||
args_array->add(name);
|
||||
|
||||
map.add("Arguments", std::move(args_array));
|
||||
|
||||
if (!arguments.empty())
|
||||
{
|
||||
auto args_pos_array = std::make_unique<JSONBuilder::JSONArray>();
|
||||
for (auto pos : arguments)
|
||||
args_pos_array->add(pos);
|
||||
|
||||
map.add("Argument Positions", std::move(args_pos_array));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -14,8 +14,7 @@ struct AggregateDescription
|
||||
{
|
||||
AggregateFunctionPtr function;
|
||||
Array parameters; /// Parameters of the (parametric) aggregate function.
|
||||
ColumnNumbers arguments;
|
||||
Names argument_names; /// used if no `arguments` are specified.
|
||||
Names argument_names;
|
||||
String column_name; /// What name to use for a column with aggregate function values
|
||||
|
||||
void explain(WriteBuffer & out, size_t indent) const; /// Get description for EXPLAIN query.
|
||||
@ -23,5 +22,4 @@ struct AggregateDescription
|
||||
};
|
||||
|
||||
using AggregateDescriptions = std::vector<AggregateDescription>;
|
||||
|
||||
}
|
||||
|
@ -260,6 +260,14 @@ auto constructWithReserveIfPossible(size_t size_hint)
|
||||
else
|
||||
return std::make_unique<Method>();
|
||||
}
|
||||
|
||||
DB::ColumnNumbers calculateKeysPositions(const DB::Block & header, const DB::Aggregator::Params & params)
|
||||
{
|
||||
DB::ColumnNumbers keys_positions(params.keys_size);
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
keys_positions[i] = header.getPositionByName(params.keys[i]);
|
||||
return keys_positions;
|
||||
}
|
||||
}
|
||||
|
||||
namespace DB
|
||||
@ -356,21 +364,17 @@ Aggregator::Params::StatsCollectingParams::StatsCollectingParams(
|
||||
|
||||
Block Aggregator::getHeader(bool final) const
|
||||
{
|
||||
return params.getHeader(final);
|
||||
return params.getHeader(header, final);
|
||||
}
|
||||
|
||||
Block Aggregator::Params::getHeader(
|
||||
const Block & src_header,
|
||||
const Block & intermediate_header,
|
||||
const ColumnNumbers & keys,
|
||||
const AggregateDescriptions & aggregates,
|
||||
bool final)
|
||||
const Block & header, bool only_merge, const Names & keys, const AggregateDescriptions & aggregates, bool final)
|
||||
{
|
||||
Block res;
|
||||
|
||||
if (intermediate_header)
|
||||
if (only_merge)
|
||||
{
|
||||
res = intermediate_header.cloneEmpty();
|
||||
res = header.cloneEmpty();
|
||||
|
||||
if (final)
|
||||
{
|
||||
@ -386,14 +390,14 @@ Block Aggregator::Params::getHeader(
|
||||
else
|
||||
{
|
||||
for (const auto & key : keys)
|
||||
res.insert(src_header.safeGetByPosition(key).cloneEmpty());
|
||||
res.insert(header.getByName(key).cloneEmpty());
|
||||
|
||||
for (const auto & aggregate : aggregates)
|
||||
{
|
||||
size_t arguments_size = aggregate.arguments.size();
|
||||
size_t arguments_size = aggregate.argument_names.size();
|
||||
DataTypes argument_types(arguments_size);
|
||||
for (size_t j = 0; j < arguments_size; ++j)
|
||||
argument_types[j] = src_header.safeGetByPosition(aggregate.arguments[j]).type;
|
||||
argument_types[j] = header.getByName(aggregate.argument_names[j]).type;
|
||||
|
||||
DataTypePtr type;
|
||||
if (final)
|
||||
@ -434,9 +438,6 @@ Aggregator::AggregateColumnsConstData Aggregator::Params::makeAggregateColumnsDa
|
||||
void Aggregator::Params::explain(WriteBuffer & out, size_t indent) const
|
||||
{
|
||||
Strings res;
|
||||
const auto & header = src_header ? src_header
|
||||
: intermediate_header;
|
||||
|
||||
String prefix(indent, ' ');
|
||||
|
||||
{
|
||||
@ -444,16 +445,13 @@ void Aggregator::Params::explain(WriteBuffer & out, size_t indent) const
|
||||
out << prefix << "Keys: ";
|
||||
|
||||
bool first = true;
|
||||
for (auto key : keys)
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
if (!first)
|
||||
out << ", ";
|
||||
first = false;
|
||||
|
||||
if (key >= header.columns())
|
||||
out << "unknown position " << key;
|
||||
else
|
||||
out << header.getByPosition(key).name;
|
||||
out << key;
|
||||
}
|
||||
|
||||
out << '\n';
|
||||
@ -470,18 +468,10 @@ void Aggregator::Params::explain(WriteBuffer & out, size_t indent) const
|
||||
|
||||
void Aggregator::Params::explain(JSONBuilder::JSONMap & map) const
|
||||
{
|
||||
const auto & header = src_header ? src_header
|
||||
: intermediate_header;
|
||||
|
||||
auto keys_array = std::make_unique<JSONBuilder::JSONArray>();
|
||||
|
||||
for (auto key : keys)
|
||||
{
|
||||
if (key >= header.columns())
|
||||
keys_array->add("");
|
||||
else
|
||||
keys_array->add(header.getByPosition(key).name);
|
||||
}
|
||||
for (const auto & key : keys)
|
||||
keys_array->add(key);
|
||||
|
||||
map.add("Keys", std::move(keys_array));
|
||||
|
||||
@ -526,7 +516,8 @@ public:
|
||||
|
||||
#endif
|
||||
|
||||
Aggregator::Aggregator(const Params & params_) : params(params_)
|
||||
Aggregator::Aggregator(const Block & header_, const Params & params_)
|
||||
: header(header_), keys_positions(calculateKeysPositions(header, params_)), params(params_)
|
||||
{
|
||||
/// Use query-level memory tracker
|
||||
if (auto * memory_tracker_child = CurrentThread::getMemoryTracker())
|
||||
@ -672,9 +663,9 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod()
|
||||
bool has_nullable_key = false;
|
||||
bool has_low_cardinality = false;
|
||||
|
||||
for (const auto & pos : params.keys)
|
||||
for (const auto & key : params.keys)
|
||||
{
|
||||
DataTypePtr type = (params.src_header ? params.src_header : params.intermediate_header).safeGetByPosition(pos).type;
|
||||
DataTypePtr type = header.getByName(key).type;
|
||||
|
||||
if (type->lowCardinality())
|
||||
{
|
||||
@ -1277,11 +1268,15 @@ void NO_INLINE Aggregator::mergeOnIntervalWithoutKeyImpl(
|
||||
}
|
||||
|
||||
|
||||
void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns & aggregate_columns, Columns & materialized_columns,
|
||||
AggregateFunctionInstructions & aggregate_functions_instructions, NestedColumnsHolder & nested_columns_holder) const
|
||||
void Aggregator::prepareAggregateInstructions(
|
||||
Columns columns,
|
||||
AggregateColumns & aggregate_columns,
|
||||
Columns & materialized_columns,
|
||||
AggregateFunctionInstructions & aggregate_functions_instructions,
|
||||
NestedColumnsHolder & nested_columns_holder) const
|
||||
{
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
aggregate_columns[i].resize(params.aggregates[i].arguments.size());
|
||||
aggregate_columns[i].resize(params.aggregates[i].argument_names.size());
|
||||
|
||||
aggregate_functions_instructions.resize(params.aggregates_size + 1);
|
||||
aggregate_functions_instructions[params.aggregates_size].that = nullptr;
|
||||
@ -1293,7 +1288,8 @@ void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns
|
||||
|
||||
for (size_t j = 0; j < aggregate_columns[i].size(); ++j)
|
||||
{
|
||||
materialized_columns.push_back(columns.at(params.aggregates[i].arguments[j])->convertToFullColumnIfConst());
|
||||
const auto pos = header.getPositionByName(params.aggregates[i].argument_names[j]);
|
||||
materialized_columns.push_back(columns.at(pos)->convertToFullColumnIfConst());
|
||||
aggregate_columns[i][j] = materialized_columns.back().get();
|
||||
|
||||
auto full_column = allow_sparse_arguments
|
||||
@ -1382,7 +1378,7 @@ bool Aggregator::executeOnBlock(Columns columns,
|
||||
/// Remember the columns we will work with
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
{
|
||||
materialized_columns.push_back(recursiveRemoveSparse(columns.at(params.keys[i]))->convertToFullColumnIfConst());
|
||||
materialized_columns.push_back(recursiveRemoveSparse(columns.at(keys_positions[i]))->convertToFullColumnIfConst());
|
||||
key_columns[i] = materialized_columns.back().get();
|
||||
|
||||
if (!result.isLowCardinality())
|
||||
@ -1954,11 +1950,11 @@ Block Aggregator::prepareBlockAndFill(
|
||||
MutableColumns final_aggregate_columns(params.aggregates_size);
|
||||
AggregateColumnsData aggregate_columns_data(params.aggregates_size);
|
||||
|
||||
Block header = getHeader(final);
|
||||
Block res_header = getHeader(final);
|
||||
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
{
|
||||
key_columns[i] = header.safeGetByPosition(i).type->createColumn();
|
||||
key_columns[i] = res_header.safeGetByPosition(i).type->createColumn();
|
||||
key_columns[i]->reserve(rows);
|
||||
}
|
||||
|
||||
@ -1967,7 +1963,7 @@ Block Aggregator::prepareBlockAndFill(
|
||||
if (!final)
|
||||
{
|
||||
const auto & aggregate_column_name = params.aggregates[i].column_name;
|
||||
aggregate_columns[i] = header.getByName(aggregate_column_name).type->createColumn();
|
||||
aggregate_columns[i] = res_header.getByName(aggregate_column_name).type->createColumn();
|
||||
|
||||
/// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states.
|
||||
ColumnAggregateFunction & column_aggregate_func = assert_cast<ColumnAggregateFunction &>(*aggregate_columns[i]);
|
||||
@ -2003,7 +1999,7 @@ Block Aggregator::prepareBlockAndFill(
|
||||
|
||||
filler(key_columns, aggregate_columns_data, final_aggregate_columns, final);
|
||||
|
||||
Block res = header.cloneEmpty();
|
||||
Block res = res_header.cloneEmpty();
|
||||
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
res.getByPosition(i).column = std::move(key_columns[i]);
|
||||
@ -2018,7 +2014,7 @@ Block Aggregator::prepareBlockAndFill(
|
||||
}
|
||||
|
||||
/// Change the size of the columns-constants in the block.
|
||||
size_t columns = header.columns();
|
||||
size_t columns = res_header.columns();
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
if (isColumnConst(*res.getByPosition(i).column))
|
||||
res.getByPosition(i).column = res.getByPosition(i).column->cut(0, rows);
|
||||
|
@ -897,13 +897,8 @@ public:
|
||||
|
||||
struct Params
|
||||
{
|
||||
/// Data structure of source blocks.
|
||||
Block src_header;
|
||||
/// Data structure of intermediate blocks before merge.
|
||||
Block intermediate_header;
|
||||
|
||||
/// What to count.
|
||||
const ColumnNumbers keys;
|
||||
const Names keys;
|
||||
const AggregateDescriptions aggregates;
|
||||
const size_t keys_size;
|
||||
const size_t aggregates_size;
|
||||
@ -937,6 +932,8 @@ public:
|
||||
bool compile_aggregate_expressions;
|
||||
size_t min_count_to_compile_aggregate_expression;
|
||||
|
||||
bool only_merge;
|
||||
|
||||
struct StatsCollectingParams
|
||||
{
|
||||
StatsCollectingParams();
|
||||
@ -957,8 +954,7 @@ public:
|
||||
StatsCollectingParams stats_collecting_params;
|
||||
|
||||
Params(
|
||||
const Block & src_header_,
|
||||
const ColumnNumbers & keys_,
|
||||
const Names & keys_,
|
||||
const AggregateDescriptions & aggregates_,
|
||||
bool overflow_row_,
|
||||
size_t max_rows_to_group_by_,
|
||||
@ -972,11 +968,9 @@ public:
|
||||
size_t min_free_disk_space_,
|
||||
bool compile_aggregate_expressions_,
|
||||
size_t min_count_to_compile_aggregate_expression_,
|
||||
const Block & intermediate_header_ = {},
|
||||
bool only_merge_ = false, // true for projections
|
||||
const StatsCollectingParams & stats_collecting_params_ = {})
|
||||
: src_header(src_header_)
|
||||
, intermediate_header(intermediate_header_)
|
||||
, keys(keys_)
|
||||
: keys(keys_)
|
||||
, aggregates(aggregates_)
|
||||
, keys_size(keys.size())
|
||||
, aggregates_size(aggregates.size())
|
||||
@ -992,33 +986,22 @@ public:
|
||||
, min_free_disk_space(min_free_disk_space_)
|
||||
, compile_aggregate_expressions(compile_aggregate_expressions_)
|
||||
, min_count_to_compile_aggregate_expression(min_count_to_compile_aggregate_expression_)
|
||||
, only_merge(only_merge_)
|
||||
, stats_collecting_params(stats_collecting_params_)
|
||||
{
|
||||
}
|
||||
|
||||
/// Only parameters that matter during merge.
|
||||
Params(
|
||||
const Block & intermediate_header_,
|
||||
const ColumnNumbers & keys_,
|
||||
const AggregateDescriptions & aggregates_,
|
||||
bool overflow_row_,
|
||||
size_t max_threads_)
|
||||
: Params(Block(), keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, 0, 0, 0, false, nullptr, max_threads_, 0, false, 0, {}, {})
|
||||
Params(const Names & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_threads_)
|
||||
: Params(
|
||||
keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, 0, 0, 0, false, nullptr, max_threads_, 0, false, 0, true, {})
|
||||
{
|
||||
intermediate_header = intermediate_header_;
|
||||
}
|
||||
|
||||
static Block getHeader(
|
||||
const Block & src_header,
|
||||
const Block & intermediate_header,
|
||||
const ColumnNumbers & keys,
|
||||
const AggregateDescriptions & aggregates,
|
||||
bool final);
|
||||
static Block
|
||||
getHeader(const Block & header, bool only_merge, const Names & keys, const AggregateDescriptions & aggregates, bool final);
|
||||
|
||||
Block getHeader(bool final) const
|
||||
{
|
||||
return getHeader(src_header, intermediate_header, keys, aggregates, final);
|
||||
}
|
||||
Block getHeader(const Block & header_, bool final) const { return getHeader(header_, only_merge, keys, aggregates, final); }
|
||||
|
||||
/// Remember the columns we will work with
|
||||
ColumnRawPtrs makeRawKeyColumns(const Block & block) const;
|
||||
@ -1029,7 +1012,7 @@ public:
|
||||
void explain(JSONBuilder::JSONMap & map) const;
|
||||
};
|
||||
|
||||
explicit Aggregator(const Params & params_);
|
||||
explicit Aggregator(const Block & header_, const Params & params_);
|
||||
|
||||
/// Process one block. Return false if the processing should be aborted (with group_by_overflow_mode = 'break').
|
||||
bool executeOnBlock(const Block & block,
|
||||
@ -1106,6 +1089,10 @@ private:
|
||||
friend class ConvertingAggregatedToChunksSource;
|
||||
friend class AggregatingInOrderTransform;
|
||||
|
||||
/// Data structure of source blocks.
|
||||
Block header;
|
||||
/// Positions of aggregation key columns in the header.
|
||||
const ColumnNumbers keys_positions;
|
||||
Params params;
|
||||
|
||||
AggregatedDataVariants::Type method_chosen;
|
||||
|
@ -40,7 +40,7 @@ struct AsynchronousMetricLogElement
|
||||
return "event_date Date CODEC(Delta(2), ZSTD(1)), "
|
||||
"event_time DateTime CODEC(Delta(4), ZSTD(1)), "
|
||||
"metric LowCardinality(String) CODEC(ZSTD(1)), "
|
||||
"value Float64 CODEC(Gorilla, ZSTD(3))";
|
||||
"value Float64 CODEC(ZSTD(3))";
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -615,8 +615,8 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti
|
||||
auto caches = FileCacheFactory::instance().getAll();
|
||||
for (const auto & [_, cache_data] : caches)
|
||||
{
|
||||
new_values["FilesystemCacheBytes"] = cache_data.cache->getUsedCacheSize();
|
||||
new_values["FilesystemCacheFiles"] = cache_data.cache->getFileSegmentsNum();
|
||||
new_values["FilesystemCacheBytes"] = cache_data->cache->getUsedCacheSize();
|
||||
new_values["FilesystemCacheFiles"] = cache_data->cache->getFileSegmentsNum();
|
||||
}
|
||||
}
|
||||
|
||||
|
60
src/Interpreters/InterpreterDescribeCacheQuery.cpp
Normal file
60
src/Interpreters/InterpreterDescribeCacheQuery.cpp
Normal file
@ -0,0 +1,60 @@
|
||||
#include <Interpreters/InterpreterDescribeCacheQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include <Parsers/ASTDescribeCacheQuery.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Common/FileCacheFactory.h>
|
||||
#include <Common/IFileCache.h>
|
||||
#include <Access/Common/AccessFlags.h>
|
||||
#include <Core/Block.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static Block getSampleBlock()
|
||||
{
|
||||
ColumnsWithTypeAndName columns{
|
||||
ColumnWithTypeAndName{std::make_shared<DataTypeUInt64>(), "max_size"},
|
||||
ColumnWithTypeAndName{std::make_shared<DataTypeUInt64>(), "max_elements"},
|
||||
ColumnWithTypeAndName{std::make_shared<DataTypeUInt64>(), "max_file_segment_size"},
|
||||
ColumnWithTypeAndName{std::make_shared<DataTypeNumber<UInt8>>(), "cache_on_write_operations"},
|
||||
ColumnWithTypeAndName{std::make_shared<DataTypeNumber<UInt8>>(), "enable_cache_hits_threshold"},
|
||||
ColumnWithTypeAndName{std::make_shared<DataTypeUInt64>(), "current_size"},
|
||||
ColumnWithTypeAndName{std::make_shared<DataTypeUInt64>(), "current_elements"},
|
||||
ColumnWithTypeAndName{std::make_shared<DataTypeString>(), "path"}
|
||||
};
|
||||
return Block(columns);
|
||||
}
|
||||
|
||||
BlockIO InterpreterDescribeCacheQuery::execute()
|
||||
{
|
||||
getContext()->checkAccess(AccessType::SHOW_CACHES);
|
||||
|
||||
const auto & ast = query_ptr->as<ASTDescribeCacheQuery &>();
|
||||
Block sample_block = getSampleBlock();
|
||||
MutableColumns res_columns = sample_block.cloneEmptyColumns();
|
||||
|
||||
auto cache_data = FileCacheFactory::instance().getByName(ast.cache_name);
|
||||
const auto & settings = cache_data.settings;
|
||||
const auto & cache = cache_data.cache;
|
||||
|
||||
res_columns[0]->insert(settings.max_size);
|
||||
res_columns[1]->insert(settings.max_elements);
|
||||
res_columns[2]->insert(settings.max_file_segment_size);
|
||||
res_columns[3]->insert(settings.cache_on_write_operations);
|
||||
res_columns[4]->insert(settings.enable_cache_hits_threshold);
|
||||
res_columns[5]->insert(cache->getUsedCacheSize());
|
||||
res_columns[6]->insert(cache->getFileSegmentsNum());
|
||||
res_columns[7]->insert(cache->getBasePath());
|
||||
|
||||
BlockIO res;
|
||||
size_t num_rows = res_columns[0]->size();
|
||||
auto source = std::make_shared<SourceFromSingleChunk>(sample_block, Chunk(std::move(res_columns), num_rows));
|
||||
res.pipeline = QueryPipeline(std::move(source));
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
22
src/Interpreters/InterpreterDescribeCacheQuery.h
Normal file
22
src/Interpreters/InterpreterDescribeCacheQuery.h
Normal file
@ -0,0 +1,22 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class InterpreterDescribeCacheQuery : public IInterpreter, WithContext
|
||||
{
|
||||
public:
|
||||
InterpreterDescribeCacheQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {}
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
};
|
||||
|
||||
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user