Merge branch 'master' into mvcc_prototype

This commit is contained in:
Alexander Tokmakov 2022-03-07 17:40:15 +01:00
commit 8acfb8d27f
458 changed files with 5016 additions and 1497 deletions

View File

@ -33,6 +33,8 @@ Checks: '-*,
performance-no-automatic-move,
performance-trivially-destructible,
performance-unnecessary-copy-initialization,
performance-noexcept-move-constructor,
performance-move-const-arg,
readability-avoid-const-params-in-decls,
readability-const-return-type,
@ -206,3 +208,5 @@ CheckOptions:
value: CamelCase
- key: modernize-loop-convert.UseCxx20ReverseRanges
value: false
- key: performance-move-const-arg.CheckTriviallyCopyableMove
value: false

View File

@ -10,6 +10,6 @@ ClickHouse® is an open-source column-oriented database management system that a
* [YouTube channel](https://www.youtube.com/c/ClickHouseDB) has a lot of content about ClickHouse in video format.
* [Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-rxm3rdrk-lIUmhLC3V8WTaL0TGxsOmg) and [Telegram](https://telegram.me/clickhouse_en) allow chatting with ClickHouse users in real-time.
* [Blog](https://clickhouse.com/blog/en/) contains various ClickHouse-related articles, as well as announcements and reports about events.
* [Code Browser (Woboq)](https://clickhouse.com/codebrowser/html_report/ClickHouse/index.html) with syntax highlight and navigation.
* [Code Browser (Woboq)](https://clickhouse.com/codebrowser/ClickHouse/index.html) with syntax highlight and navigation.
* [Code Browser (github.dev)](https://github.dev/ClickHouse/ClickHouse) with syntax highlight, powered by github.dev.
* [Contacts](https://clickhouse.com/company/#contact) can help to get your questions answered if there are any.

View File

@ -17,6 +17,7 @@ set (SRCS
terminalColors.cpp
errnoToString.cpp
StringRef.cpp
safeExit.cpp
)
if (ENABLE_REPLXX)

View File

@ -19,6 +19,12 @@
#if defined(__SSE4_2__)
#include <smmintrin.h>
#include <nmmintrin.h>
#define CRC_INT _mm_crc32_u64
#endif
#if defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
#include <arm_acle.h>
#define CRC_INT __crc32cd
#endif
@ -205,7 +211,7 @@ struct StringRefHash64
}
};
#if defined(__SSE4_2__)
#if defined(CRC_INT)
/// Parts are taken from CityHash.
@ -281,13 +287,13 @@ struct CRC32Hash
do
{
UInt64 word = unalignedLoad<UInt64>(pos);
res = _mm_crc32_u64(res, word);
res = CRC_INT(res, word);
pos += 8;
} while (pos + 8 < end);
UInt64 word = unalignedLoad<UInt64>(end - 8); /// I'm not sure if this is normal.
res = _mm_crc32_u64(res, word);
res = CRC_INT(res, word);
return res;
}

View File

@ -26,3 +26,27 @@ void insertAtEnd(std::vector<T> & dest, std::vector<T> && src)
dest.insert(dest.end(), std::make_move_iterator(src.begin()), std::make_move_iterator(src.end()));
src.clear();
}
template <typename Container>
void insertAtEnd(Container & dest, const Container & src)
{
if (src.empty())
return;
dest.insert(dest.end(), src.begin(), src.end());
}
template <typename Container>
void insertAtEnd(Container & dest, Container && src)
{
if (src.empty())
return;
if (dest.empty())
{
dest.swap(src);
return;
}
dest.insert(dest.end(), std::make_move_iterator(src.begin()), std::make_move_iterator(src.end()));
src.clear();
}

18
base/base/safeExit.cpp Normal file
View File

@ -0,0 +1,18 @@
#if defined(OS_LINUX)
# include <sys/syscall.h>
#endif
#include <unistd.h>
#include <base/safeExit.h>
#include <base/defines.h>
[[noreturn]] void safeExit(int code)
{
#if defined(THREAD_SANITIZER) && defined(OS_LINUX)
/// Thread sanitizer tries to do something on exit that we don't need if we want to exit immediately,
/// while connection handling threads are still run.
(void)syscall(SYS_exit_group, code);
__builtin_unreachable();
#else
_exit(code);
#endif
}

4
base/base/safeExit.h Normal file
View File

@ -0,0 +1,4 @@
#pragma once
/// _exit() with a workaround for TSan.
[[noreturn]] void safeExit(int code);

View File

@ -11,10 +11,6 @@ DATASET="${TABLE}_v1.tar.xz"
QUERIES_FILE="queries.sql"
TRIES=3
AMD64_BIN_URL="https://builds.clickhouse.com/master/amd64/clickhouse"
AARCH64_BIN_URL="https://builds.clickhouse.com/master/aarch64/clickhouse"
POWERPC64_BIN_URL="https://builds.clickhouse.com/master/ppc64le/clickhouse"
# Note: on older Ubuntu versions, 'axel' does not support IPv6. If you are using IPv6-only servers on very old Ubuntu, just don't install 'axel'.
FASTER_DOWNLOAD=wget
@ -33,20 +29,60 @@ fi
mkdir -p clickhouse-benchmark-$SCALE
pushd clickhouse-benchmark-$SCALE
if [[ ! -f clickhouse ]]; then
CPU=$(uname -m)
if [[ ($CPU == x86_64) || ($CPU == amd64) ]]; then
$FASTER_DOWNLOAD "$AMD64_BIN_URL"
elif [[ $CPU == aarch64 ]]; then
$FASTER_DOWNLOAD "$AARCH64_BIN_URL"
elif [[ $CPU == powerpc64le ]]; then
$FASTER_DOWNLOAD "$POWERPC64_BIN_URL"
else
echo "Unsupported CPU type: $CPU"
exit 1
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
@ -88,7 +124,12 @@ echo
cat "$QUERIES_FILE" | sed "s/{table}/${TABLE}/g" | while read query; do
sync
echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null
if [ "${OS}" = "Darwin" ]
then
sudo purge > /dev/null
else
echo 3 | sudo tee /proc/sys/vm/drop_caches >/dev/null
fi
echo -n "["
for i in $(seq 1 $TRIES); do
@ -104,27 +145,45 @@ echo
echo "Benchmark complete. System info:"
echo
echo '----Version, build id-----------'
./clickhouse local --query "SELECT format('Version: {}, build id: {}', version(), buildId())"
./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw
./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))"
echo '----CPU-------------------------'
cat /proc/cpuinfo | grep -i -F 'model name' | uniq
lscpu
echo '----Block Devices---------------'
lsblk
echo '----Disk Free and Total--------'
df -h .
echo '----Memory Free and Total-------'
free -h
echo '----Physical Memory Amount------'
cat /proc/meminfo | grep MemTotal
echo '----RAID Info-------------------'
cat /proc/mdstat
#echo '----PCI-------------------------'
#lspci
#echo '----All Hardware Info-----------'
#lshw
echo '--------------------------------'
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'
echo '----Disk Free and Total--------'
df -h .
echo '----Memory Free and Total-------'
vm_stat
echo '----Physical Memory Amount------'
ls -l /var/vm
echo '--------------------------------'
else
echo '----Version, build id-----------'
./clickhouse local --query "SELECT format('Version: {}, build id: {}', version(), buildId())"
./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw
./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))"
echo '----CPU-------------------------'
cat /proc/cpuinfo | grep -i -F 'model name' | uniq
lscpu
echo '----Block Devices---------------'
lsblk
echo '----Disk Free and Total--------'
df -h .
echo '----Memory Free and Total-------'
free -h
echo '----Physical Memory Amount------'
cat /proc/meminfo | grep MemTotal
echo '----RAID Info-------------------'
cat /proc/mdstat
#echo '----PCI-------------------------'
#lspci
#echo '----All Hardware Info-----------'
#lshw
echo '--------------------------------'
fi
echo

View File

@ -55,5 +55,5 @@ endif ()
if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS)
message(STATUS
"${CMAKE_CURRENT_SOURCE_DIR}: Have ${AVAILABLE_PHYSICAL_MEMORY} megabytes of memory.
Limiting concurrent linkers jobs to ${PARALLEL_LINK_JOBS} and compiler jobs to ${PARALLEL_COMPILE_JOBS}")
Limiting concurrent linkers jobs to ${PARALLEL_LINK_JOBS} and compiler jobs to ${PARALLEL_COMPILE_JOBS} (system has ${NUMBER_OF_LOGICAL_CORES} logical cores)")
endif ()

2
contrib/jemalloc vendored

@ -1 +1 @@
Subproject commit ca709c3139f77f4c00a903cdee46d71e9028f6c6
Subproject commit 78b58379c854a639df79beb3289351129d863d4b

View File

@ -4,7 +4,7 @@ FROM ubuntu:20.04
ARG apt_archive="http://archive.ubuntu.com"
RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list
ARG repository="deb https://repo.clickhouse.com/deb/stable/ main/"
ARG repository="deb https://packages.clickhouse.com/deb stable main"
ARG version=22.1.1.*
# set non-empty deb_location_url url to create a docker image
@ -58,7 +58,7 @@ RUN groupadd -r clickhouse --gid=101 \
wget \
tzdata \
&& mkdir -p /etc/apt/sources.list.d \
&& apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 \
&& apt-key adv --keyserver keyserver.ubuntu.com --recv 8919F6BD2B48D754 \
&& echo $repository > /etc/apt/sources.list.d/clickhouse.list \
&& if [ -n "$deb_location_url" ]; then \
echo "installing from custom url with deb packages: $deb_location_url" \

View File

@ -263,9 +263,20 @@ function run_tests
if [[ $NPROC == 0 ]]; then
NPROC=1
fi
time clickhouse-test --hung-check -j "${NPROC}" --order=random \
--fast-tests-only --no-long --testname --shard --zookeeper --check-zookeeper-session \
-- "$FASTTEST_FOCUS" 2>&1 \
local test_opts=(
--hung-check
--fast-tests-only
--no-long
--testname
--shard
--zookeeper
--check-zookeeper-session
--order random
--print-time
--jobs "${NPROC}"
)
time clickhouse-test "${test_opts[@]}" -- "$FASTTEST_FOCUS" 2>&1 \
| ts '%Y-%m-%d %H:%M:%S' \
| tee "$FASTTEST_OUTPUT/test_result.txt"
set -e

View File

@ -42,6 +42,9 @@ COPY prepare_hive_data.sh /
COPY demo_data.txt /
ENV PATH=/apache-hive-2.3.9-bin/bin:/hadoop-3.1.0/bin:/hadoop-3.1.0/sbin:$PATH
RUN service ssh start && sed s/HOSTNAME/$HOSTNAME/ /hadoop-3.1.0/etc/hadoop/core-site.xml.template > /hadoop-3.1.0/etc/hadoop/core-site.xml && hdfs namenode -format
RUN apt install -y python3 python3-pip
RUN pip3 install flask requests
COPY http_api_server.py /
COPY start.sh /

View File

@ -0,0 +1,70 @@
import os
import subprocess
import datetime
from flask import Flask, flash, request, redirect, url_for
def run_command(command, wait=False):
print("{} - execute shell command:{}".format(datetime.datetime.now(), command))
lines = []
p = subprocess.Popen(command,
stdout=subprocess.PIPE,
stderr=subprocess.STDOUT,
shell=True)
if wait:
for l in iter(p.stdout.readline, b''):
lines.append(l)
p.poll()
return (lines, p.returncode)
else:
return(iter(p.stdout.readline, b''), 0)
UPLOAD_FOLDER = './'
ALLOWED_EXTENSIONS = {'txt', 'sh'}
app = Flask(__name__)
app.config['UPLOAD_FOLDER'] = UPLOAD_FOLDER
@app.route('/')
def hello_world():
return 'Hello World'
def allowed_file(filename):
return '.' in filename and \
filename.rsplit('.', 1)[1].lower() in ALLOWED_EXTENSIONS
@app.route('/upload', methods=['GET', 'POST'])
def upload_file():
if request.method == 'POST':
# check if the post request has the file part
if 'file' not in request.files:
flash('No file part')
return redirect(request.url)
file = request.files['file']
# If the user does not select a file, the browser submits an
# empty file without a filename.
if file.filename == '':
flash('No selected file')
return redirect(request.url)
if file and allowed_file(file.filename):
filename = file.filename
file.save(os.path.join(app.config['UPLOAD_FOLDER'], filename))
return redirect(url_for('upload_file', name=filename))
return '''
<!doctype html>
<title>Upload new File</title>
<h1>Upload new File</h1>
<form method=post enctype=multipart/form-data>
<input type=file name=file>
<input type=submit value=Upload>
</form>
'''
@app.route('/run', methods=['GET', 'POST'])
def parse_request():
data = request.data # data is empty
run_command(data, wait=True)
return 'Ok'
if __name__ == '__main__':
app.run(port=5011)

View File

@ -2,5 +2,9 @@
hive -e "create database test"
hive -e "create table test.demo(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'; create table test.demo_orc(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'; "
hive -e "create table test.parquet_demo(id string, score int) PARTITIONED BY(day string, hour string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'"
hive -e "create table test.demo_text(id string, score int, day string)row format delimited fields terminated by ','; load data local inpath '/demo_data.txt' into table test.demo_text "
hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.demo partition(day) select * from test.demo_text; insert into test.demo_orc partition(day) select * from test.demo_text"
hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.demo partition(day) select * from test.demo_text; insert into test.demo_orc partition(day) select * from test.demo_text"
hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '00' as hour from test.demo;"
hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '01' as hour from test.demo;"

View File

@ -1,6 +1,5 @@
service ssh start
sed s/HOSTNAME/$HOSTNAME/ /hadoop-3.1.0/etc/hadoop/core-site.xml.template > /hadoop-3.1.0/etc/hadoop/core-site.xml
hadoop namenode -format
start-all.sh
service mysql start
mysql -u root -e "CREATE USER \"test\"@\"localhost\" IDENTIFIED BY \"test\""
@ -9,4 +8,4 @@ schematool -initSchema -dbType mysql
#nohup hiveserver2 &
nohup hive --service metastore &
bash /prepare_hive_data.sh
while true; do sleep 1000; done
python3 http_api_server.py

View File

@ -1,7 +1,7 @@
sudo apt-get install apt-transport-https ca-certificates dirmngr
sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv E0C56BD4
sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754
echo "deb https://repo.clickhouse.com/deb/stable/ main/" | sudo tee \
echo "deb https://packages.clickhouse.com/deb stable main/" | sudo tee \
/etc/apt/sources.list.d/clickhouse.list
sudo apt-get update

View File

@ -0,0 +1,11 @@
sudo apt-get install apt-transport-https ca-certificates dirmngr
sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv E0C56BD4
echo "deb https://repo.clickhouse.com/deb/stable/ main/" | sudo tee \
/etc/apt/sources.list.d/clickhouse.list
sudo apt-get update
sudo apt-get install -y clickhouse-server clickhouse-client
sudo service clickhouse-server start
clickhouse-client # or "clickhouse-client --password" if you set up a password.

View File

@ -1,7 +1,6 @@
sudo yum install yum-utils
sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG
sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/clickhouse.repo
sudo yum install clickhouse-server clickhouse-client
sudo yum install -y yum-utils
sudo yum-config-manager --add-repo https://packages.clickhouse.com/rpm/clickhouse.repo
sudo yum install -y clickhouse-server clickhouse-client
sudo /etc/init.d/clickhouse-server start
clickhouse-client # or "clickhouse-client --password" if you set up a password.

View File

@ -0,0 +1,7 @@
sudo yum install yum-utils
sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG
sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/clickhouse.repo
sudo yum install clickhouse-server clickhouse-client
sudo /etc/init.d/clickhouse-server start
clickhouse-client # or "clickhouse-client --password" if you set up a password.

View File

@ -1,19 +1,20 @@
export LATEST_VERSION=$(curl -s https://repo.clickhouse.com/tgz/stable/ | \
LATEST_VERSION=$(curl -s https://packages.clickhouse.com/tgz/stable/ | \
grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | sort -V -r | head -n 1)
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-dbg-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-server-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-client-$LATEST_VERSION.tgz
export LATEST_VERSION
curl -O "https://packages.clickhouse.com/tgz/stable/clickhouse-common-static-$LATEST_VERSION.tgz"
curl -O "https://packages.clickhouse.com/tgz/stable/clickhouse-common-static-dbg-$LATEST_VERSION.tgz"
curl -O "https://packages.clickhouse.com/tgz/stable/clickhouse-server-$LATEST_VERSION.tgz"
curl -O "https://packages.clickhouse.com/tgz/stable/clickhouse-client-$LATEST_VERSION.tgz"
tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz
sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh
tar -xzvf "clickhouse-common-static-$LATEST_VERSION.tgz"
sudo "clickhouse-common-static-$LATEST_VERSION/install/doinst.sh"
tar -xzvf clickhouse-common-static-dbg-$LATEST_VERSION.tgz
sudo clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh
tar -xzvf "clickhouse-common-static-dbg-$LATEST_VERSION.tgz"
sudo "clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh"
tar -xzvf clickhouse-server-$LATEST_VERSION.tgz
sudo clickhouse-server-$LATEST_VERSION/install/doinst.sh
tar -xzvf "clickhouse-server-$LATEST_VERSION.tgz"
sudo "clickhouse-server-$LATEST_VERSION/install/doinst.sh"
sudo /etc/init.d/clickhouse-server start
tar -xzvf clickhouse-client-$LATEST_VERSION.tgz
sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh
tar -xzvf "clickhouse-client-$LATEST_VERSION.tgz"
sudo "clickhouse-client-$LATEST_VERSION/install/doinst.sh"

View File

@ -0,0 +1,19 @@
export LATEST_VERSION=$(curl -s https://repo.clickhouse.com/tgz/stable/ | \
grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | sort -V -r | head -n 1)
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-dbg-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-server-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-client-$LATEST_VERSION.tgz
tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz
sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh
tar -xzvf clickhouse-common-static-dbg-$LATEST_VERSION.tgz
sudo clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh
tar -xzvf clickhouse-server-$LATEST_VERSION.tgz
sudo clickhouse-server-$LATEST_VERSION/install/doinst.sh
sudo /etc/init.d/clickhouse-server start
tar -xzvf clickhouse-client-$LATEST_VERSION.tgz
sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh

View File

@ -5,7 +5,7 @@ toc_title: Source Code Browser
# Browse ClickHouse Source Code {#browse-clickhouse-source-code}
You can use **Woboq** online code browser available [here](https://clickhouse.com/codebrowser/html_report/ClickHouse/src/index.html). It provides code navigation and semantic highlighting, search and indexing. The code snapshot is updated daily.
You can use **Woboq** online code browser available [here](https://clickhouse.com/codebrowser/ClickHouse/src/index.html). It provides code navigation and semantic highlighting, search and indexing. The code snapshot is updated daily.
Also, you can browse sources on [GitHub](https://github.com/ClickHouse/ClickHouse) as usual.

View File

@ -156,14 +156,6 @@ $ cd ClickHouse
$ ./release
```
## Faster builds for development
Normally all tools of the ClickHouse bundle, such as `clickhouse-server`, `clickhouse-client` etc., are linked into a single static executable, `clickhouse`. This executable must be re-linked on every change, which might be slow. One common way to improve build time is to use the 'split' build configuration, which builds a separate binary for every tool, and further splits the code into several shared libraries. To enable this tweak, pass the following flags to `cmake`:
```
-DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1
```
## You Dont Have to Build ClickHouse {#you-dont-have-to-build-clickhouse}
ClickHouse is available in pre-built binaries and packages. Binaries are portable and can be run on any Linux flavour.
@ -172,9 +164,9 @@ They are built for stable, prestable and testing releases as long as for every c
To find the freshest build from `master`, go to [commits page](https://github.com/ClickHouse/ClickHouse/commits/master), click on the first green checkmark or red cross near commit, and click to the “Details” link right after “ClickHouse Build Check”.
## Split build configuration {#split-build}
## Faster builds for development: Split build configuration {#split-build}
Normally ClickHouse is statically linked into a single static `clickhouse` binary with minimal dependencies. This is convenient for distribution, but it means that on every change the entire binary is linked again, which is slow and may be inconvenient for development. There is an alternative configuration which creates dynamically loaded shared libraries instead, allowing faster incremental builds. To use it, add the following flags to your `cmake` invocation:
Normally, ClickHouse is statically linked into a single static `clickhouse` binary with minimal dependencies. This is convenient for distribution, but it means that on every change the entire binary needs to be linked, which is slow and may be inconvenient for development. There is an alternative configuration which instead creates dynamically loaded shared libraries and separate binaries `clickhouse-server`, `clickhouse-client` etc., allowing for faster incremental builds. To use it, add the following flags to your `cmake` invocation:
```
-DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1
```

View File

@ -36,6 +36,7 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo
- `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.
```sql
CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user', '***')

View File

@ -22,4 +22,4 @@ Here is the illustration of the difference between traditional row-oriented syst
**Columnar**
![Columnar](https://clickhouse.com/docs/en/images/column-oriented.gif#)
A columnar database is a preferred choice for analytical applications because it allows to have many columns in a table just in case, but do not pay the cost for unused columns on read query execution time. Column-oriented databases are designed for big data processing because and data warehousing, they often natively scale using distributed clusters of low-cost hardware to increase throughput. ClickHouse does it with combination of [distributed](../../engines/table-engines/special/distributed.md) and [replicated](../../engines/table-engines/mergetree-family/replication.md) tables.
A columnar database is a preferred choice for analytical applications because it allows to have many columns in a table just in case, but do not pay the cost for unused columns on read query execution time. Column-oriented databases are designed for big data processing and data warehousing, because they often natively scale using distributed clusters of low-cost hardware to increase throughput. ClickHouse does it with combination of [distributed](../../engines/table-engines/special/distributed.md) and [replicated](../../engines/table-engines/mergetree-family/replication.md) tables.

View File

@ -27,9 +27,17 @@ It is recommended to use official pre-compiled `deb` packages for Debian or Ubun
{% include 'install/deb.sh' %}
```
<details markdown="1">
<summary>Deprecated Method for installing deb-packages</summary>
``` bash
{% include 'install/deb_repo.sh' %}
```
</details>
You can replace `stable` with `lts` or `testing` to use different [release trains](../faq/operations/production.md) based on your needs.
You can also download and install packages manually from [here](https://repo.clickhouse.com/deb/stable/main/).
You can also download and install packages manually from [here](https://packages.clickhouse.com/deb/pool/stable).
#### Packages {#packages}
@ -49,11 +57,17 @@ It is recommended to use official pre-compiled `rpm` packages for CentOS, RedHat
First, you need to add the official repository:
``` bash
sudo yum install yum-utils
sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG
sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64
{% include 'install/rpm.sh' %}
```
<details markdown="1">
<summary>Deprecated Method for installing rpm-packages</summary>
``` bash
{% include 'install/rpm_repo.sh' %}
```
</details>
If you want to use the most recent version, replace `stable` with `testing` (this is recommended for your testing environments). `prestable` is sometimes also available.
Then run these commands to install packages:
@ -62,36 +76,27 @@ Then run these commands to install packages:
sudo yum install clickhouse-server clickhouse-client
```
You can also download and install packages manually from [here](https://repo.clickhouse.com/rpm/stable/x86_64).
You can also download and install packages manually from [here](https://packages.clickhouse.com/rpm/stable).
### From Tgz Archives {#from-tgz-archives}
It is recommended to use official pre-compiled `tgz` archives for all Linux distributions, where installation of `deb` or `rpm` packages is not possible.
The required version can be downloaded with `curl` or `wget` from repository https://repo.clickhouse.com/tgz/.
The required version can be downloaded with `curl` or `wget` from repository https://packages.clickhouse.com/tgz/.
After that downloaded archives should be unpacked and installed with installation scripts. Example for the latest stable version:
``` bash
export LATEST_VERSION=`curl https://api.github.com/repos/ClickHouse/ClickHouse/tags 2>/dev/null | grep stable | grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | head -n 1`
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-dbg-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-server-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-client-$LATEST_VERSION.tgz
tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz
sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh
tar -xzvf clickhouse-common-static-dbg-$LATEST_VERSION.tgz
sudo clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh
tar -xzvf clickhouse-server-$LATEST_VERSION.tgz
sudo clickhouse-server-$LATEST_VERSION/install/doinst.sh
sudo /etc/init.d/clickhouse-server start
tar -xzvf clickhouse-client-$LATEST_VERSION.tgz
sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh
{% include 'install/tgz.sh' %}
```
<details markdown="1">
<summary>Deprecated Method for installing tgz archives</summary>
``` bash
{% include 'install/tgz_repo.sh' %}
```
</details>
For production environments, its recommended to use the latest `stable`-version. You can find its number on GitHub page https://github.com/ClickHouse/ClickHouse/tags with postfix `-stable`.
### From Docker Image {#from-docker-image}
@ -215,6 +220,6 @@ SELECT 1
**Congratulations, the system works!**
To continue experimenting, you can download one of the test data sets or go through [tutorial](https://clickhouse.com/tutorial.html).
To continue experimenting, you can download one of the test data sets or go through [tutorial](./tutorial.md).
[Original article](https://clickhouse.com/docs/en/getting_started/install/) <!--hide-->

View File

@ -38,6 +38,18 @@ Alternatively you can perform benchmark in the following steps.
wget https://builds.clickhouse.com/master/amd64/clickhouse
# For aarch64:
wget https://builds.clickhouse.com/master/aarch64/clickhouse
# For powerpc64le:
wget https://builds.clickhouse.com/master/powerpc64le/clickhouse
# For freebsd:
wget https://builds.clickhouse.com/master/freebsd/clickhouse
# For freebsd-aarch64:
wget https://builds.clickhouse.com/master/freebsd-aarch64/clickhouse
# For freebsd-powerpc64le:
wget https://builds.clickhouse.com/master/freebsd-powerpc64le/clickhouse
# For macos:
wget https://builds.clickhouse.com/master/macos/clickhouse
# For macos-aarch64:
wget https://builds.clickhouse.com/master/macos-aarch64/clickhouse
# Then do:
chmod a+x clickhouse
```

View File

@ -82,7 +82,7 @@ Columns:
- `path` ([String](../../sql-reference/data-types/string.md)) Absolute path to the folder with data part files.
- `disk` ([String](../../sql-reference/data-types/string.md)) Name of a disk that stores the data part.
- `disk_name` ([String](../../sql-reference/data-types/string.md)) Name of a disk that stores the data part.
- `hash_of_all_files` ([String](../../sql-reference/data-types/string.md)) [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128) of compressed files.

View File

@ -1,11 +1,11 @@
---
toc_priority: 59
toc_title: Yandex.Metrica Dictionaries
toc_title: Embedded Dictionaries
---
# Functions for Working with Yandex.Metrica Dictionaries {#functions-for-working-with-yandex-metrica-dictionaries}
# Functions for Working with Embedded Dictionaries
In order for the functions below to work, the server config must specify the paths and addresses for getting all the Yandex.Metrica dictionaries. The dictionaries are loaded at the first call of any of these functions. If the reference lists cant be loaded, an exception is thrown.
In order for the functions below to work, the server config must specify the paths and addresses for getting all the embedded dictionaries. The dictionaries are loaded at the first call of any of these functions. If the reference lists cant be loaded, an exception is thrown.
For information about creating reference lists, see the section “Dictionaries”.
@ -33,7 +33,7 @@ regionToCountry(RegionID, 'ua') Uses the dictionary for the 'ua' key: /opt/g
### regionToCity(id\[, geobase\]) {#regiontocityid-geobase}
Accepts a UInt32 number the region ID from the Yandex geobase. If this region is a city or part of a city, it returns the region ID for the appropriate city. Otherwise, returns 0.
Accepts a UInt32 number the region ID from the geobase. If this region is a city or part of a city, it returns the region ID for the appropriate city. Otherwise, returns 0.
### regionToArea(id\[, geobase\]) {#regiontoareaid-geobase}
@ -117,7 +117,7 @@ regionToTopContinent(id[, geobase])
**Arguments**
- `id` — Region ID from the Yandex geobase. [UInt32](../../sql-reference/data-types/int-uint.md).
- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md).
- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional.
**Returned value**
@ -132,7 +132,7 @@ Type: `UInt32`.
Gets the population for a region.
The population can be recorded in files with the geobase. See the section “External dictionaries”.
If the population is not recorded for the region, it returns 0.
In the Yandex geobase, the population might be recorded for child regions, but not for parent regions.
In the geobase, the population might be recorded for child regions, but not for parent regions.
### regionIn(lhs, rhs\[, geobase\]) {#regioninlhs-rhs-geobase}
@ -141,12 +141,12 @@ The relationship is reflexive any region also belongs to itself.
### regionHierarchy(id\[, geobase\]) {#regionhierarchyid-geobase}
Accepts a UInt32 number the region ID from the Yandex geobase. Returns an array of region IDs consisting of the passed region and all parents along the chain.
Accepts a UInt32 number the region ID from the geobase. Returns an array of region IDs consisting of the passed region and all parents along the chain.
Example: `regionHierarchy(toUInt32(213)) = [213,1,3,225,10001,10000]`.
### regionToName(id\[, lang\]) {#regiontonameid-lang}
Accepts a UInt32 number the region ID from the Yandex geobase. A string with the name of the language can be passed as a second argument. Supported languages are: ru, en, ua, uk, by, kz, tr. If the second argument is omitted, the language ru is used. If the language is not supported, an exception is thrown. Returns a string the name of the region in the corresponding language. If the region with the specified ID does not exist, an empty string is returned.
Accepts a UInt32 number the region ID from the geobase. A string with the name of the language can be passed as a second argument. Supported languages are: ru, en, ua, uk, by, kz, tr. If the second argument is omitted, the language ru is used. If the language is not supported, an exception is thrown. Returns a string the name of the region in the corresponding language. If the region with the specified ID does not exist, an empty string is returned.
`ua` and `uk` both mean Ukrainian.

View File

@ -11,7 +11,7 @@ ClickHouse supports the following syntax variants:
- `LIMIT [offset_value, ]n BY expressions`
- `LIMIT n OFFSET offset_value BY expressions`
During query processing, ClickHouse selects data ordered by sorting key. The sorting key is set explicitly using an [ORDER BY](../../../sql-reference/statements/select/order-by.md) clause or implicitly as a property of the table engine. Then ClickHouse applies `LIMIT n BY expressions` and returns the first `n` rows for each distinct combination of `expressions`. If `OFFSET` is specified, then for each data block that belongs to a distinct combination of `expressions`, ClickHouse skips `offset_value` number of rows from the beginning of the block and returns a maximum of `n` rows as a result. If `offset_value` is bigger than the number of rows in the data block, ClickHouse returns zero rows from the block.
During query processing, ClickHouse selects data ordered by sorting key. The sorting key is set explicitly using an [ORDER BY](order-by.md#select-order-by) clause or implicitly as a property of the table engine (row order is only guaranteed when using [ORDER BY](order-by.md#select-order-by), otherwise the row blocks will not be ordered due to multi-threading). Then ClickHouse applies `LIMIT n BY expressions` and returns the first `n` rows for each distinct combination of `expressions`. If `OFFSET` is specified, then for each data block that belongs to a distinct combination of `expressions`, ClickHouse skips `offset_value` number of rows from the beginning of the block and returns a maximum of `n` rows as a result. If `offset_value` is bigger than the number of rows in the data block, ClickHouse returns zero rows from the block.
!!! note "Note"
`LIMIT BY` is not related to [LIMIT](../../../sql-reference/statements/select/limit.md). They can both be used in the same query.

View File

@ -2968,7 +2968,7 @@ No changes compared to v20.4.3.16-stable.
* Updated checking for hung queries in clickhouse-test script [#8858](https://github.com/ClickHouse/ClickHouse/pull/8858) ([Alexander Kazakov](https://github.com/Akazz))
* Removed some useless files from repository. [#8843](https://github.com/ClickHouse/ClickHouse/pull/8843) ([alexey-milovidov](https://github.com/alexey-milovidov))
* Changed type of math perftests from `once` to `loop`. [#8783](https://github.com/ClickHouse/ClickHouse/pull/8783) ([Nikolai Kochetov](https://github.com/KochetovNicolai))
* Add docker image which allows to build interactive code browser HTML report for our codebase. [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([alesapin](https://github.com/alesapin)) See [Woboq Code Browser](https://clickhouse-test-reports.s3.yandex.net/codebrowser/html_report///ClickHouse/dbms/index.html)
* Add docker image which allows to build interactive code browser HTML report for our codebase. [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([alesapin](https://github.com/alesapin)) See [Woboq Code Browser](https://clickhouse-test-reports.s3.yandex.net/codebrowser/ClickHouse/dbms/index.html)
* Suppress some test failures under MSan. [#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alexander Kuzmenkov](https://github.com/akuzm))
* Speedup "exception while insert" test. This test often time out in debug-with-coverage build. [#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([alexey-milovidov](https://github.com/alexey-milovidov))
* Updated `libcxx` and `libcxxabi` to master. In preparation to [#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([alexey-milovidov](https://github.com/alexey-milovidov))

View File

@ -7,7 +7,7 @@ toc_title: "\u30BD\u30FC\u30B9\u30B3\u30FC\u30C9\u306E\u53C2\u7167"
# ClickHouseのソースコードを参照 {#browse-clickhouse-source-code}
以下を使用できます **Woboq** オンラインのコードブラウザをご利用 [ここに](https://clickhouse.com/codebrowser/html_report/ClickHouse/src/index.html). このコードナビゲーションや意味のハイライト表示、検索インデックス. コードのスナップショットは随時更新中です。
以下を使用できます **Woboq** オンラインのコードブラウザをご利用 [ここに](https://clickhouse.com/codebrowser/ClickHouse/src/index.html). このコードナビゲーションや意味のハイライト表示、検索インデックス. コードのスナップショットは随時更新中です。
また、ソースを参照することもできます [GitHub](https://github.com/ClickHouse/ClickHouse) いつものように

View File

@ -14,7 +14,7 @@ toc_title: "\u30C7\u30FC\u30BF\u8907\u88FD"
- レプリケートリプレースマージツリー
- 複製された集合マージツリー
- レプリケートコラプシングマージツリー
- ReplicatedVersionedCollapsingMergetree
- ReplicatedVersionedCollapsingMergeTree
- レプリケートグラフィティマージツリー
複製の作品のレベルを個別のテーブルではなく、全体のサーバーです。 サーバーでの店舗も複製、非複製のテーブルでも同時に行います。

View File

@ -28,9 +28,17 @@ Debian や Ubuntu 用にコンパイル済みの公式パッケージ `deb` を
{% include 'install/deb.sh' %}
```
<details markdown="1">
<summary>Deprecated Method for installing deb-packages</summary>
``` bash
{% include 'install/deb_repo.sh' %}
```
</details>
最新版を使いたい場合は、`stable`を`testing`に置き換えてください。(テスト環境ではこれを推奨します)
同様に、[こちら](https://repo.clickhouse.com/deb/stable/main/)からパッケージをダウンロードして、手動でインストールすることもできます。
同様に、[こちら](https://packages.clickhouse.com/deb/pool/stable)からパッケージをダウンロードして、手動でインストールすることもできます。
#### パッケージ {#packages}
@ -46,11 +54,17 @@ CentOS、RedHat、その他すべてのrpmベースのLinuxディストリビュ
まず、公式リポジトリを追加する必要があります:
``` bash
sudo yum install yum-utils
sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG
sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64
{% include 'install/rpm.sh' %}
```
<details markdown="1">
<summary>Deprecated Method for installing rpm-packages</summary>
``` bash
{% include 'install/rpm_repo.sh' %}
```
</details>
最新版を使いたい場合は `stable``testing` に置き換えてください。(テスト環境ではこれが推奨されています)。`prestable` もしばしば同様に利用できます。
そして、以下のコマンドを実行してパッケージをインストールします:
@ -59,35 +73,26 @@ sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64
sudo yum install clickhouse-server clickhouse-client
```
同様に、[こちら](https://repo.clickhouse.com/rpm/stable/x86_64) からパッケージをダウンロードして、手動でインストールすることもできます。
同様に、[こちら](https://packages.clickhouse.com/rpm/stable) からパッケージをダウンロードして、手動でインストールすることもできます。
### Tgzアーカイブから {#from-tgz-archives}
すべての Linux ディストリビューションで、`deb` や `rpm` パッケージがインストールできない場合は、公式のコンパイル済み `tgz` アーカイブを使用することをお勧めします。
必要なバージョンは、リポジトリ https://repo.clickhouse.com/tgz/ から `curl` または `wget` でダウンロードできます。その後、ダウンロードしたアーカイブを解凍し、インストールスクリプトでインストールしてください。最新版の例は以下です:
必要なバージョンは、リポジトリ https://packages.clickhouse.com/tgz/ から `curl` または `wget` でダウンロードできます。その後、ダウンロードしたアーカイブを解凍し、インストールスクリプトでインストールしてください。最新版の例は以下です:
``` bash
export LATEST_VERSION=`curl https://api.github.com/repos/ClickHouse/ClickHouse/tags 2>/dev/null | grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | head -n 1`
curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-dbg-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/clickhouse-server-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/clickhouse-client-$LATEST_VERSION.tgz
tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz
sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh
tar -xzvf clickhouse-common-static-dbg-$LATEST_VERSION.tgz
sudo clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh
tar -xzvf clickhouse-server-$LATEST_VERSION.tgz
sudo clickhouse-server-$LATEST_VERSION/install/doinst.sh
sudo /etc/init.d/clickhouse-server start
tar -xzvf clickhouse-client-$LATEST_VERSION.tgz
sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh
{% include 'install/tgz.sh' %}
```
<details markdown="1">
<summary>Deprecated Method for installing tgz archives</summary>
``` bash
{% include 'install/tgz_repo.sh' %}
```
</details>
本番環境では、最新の `stable` バージョンを使うことをお勧めします。GitHub のページ https://github.com/ClickHouse/ClickHouse/tags で 接尾辞 `-stable` となっているバージョン番号として確認できます。
### Dockerイメージから {#from-docker-image}
@ -186,6 +191,6 @@ SELECT 1
**おめでとうございます!システムが動きました!**
動作確認を続けるには、テストデータセットをダウンロードするか、[チュートリアル](https://clickhouse.com/tutorial.html)を参照してください。
動作確認を続けるには、テストデータセットをダウンロードするか、[チュートリアル](./tutorial.md)を参照してください。
[元の記事](https://clickhouse.com/docs/en/getting_started/install/) <!--hide-->

View File

@ -20,9 +20,21 @@ toc_title: "\u30CF\u30FC\u30C9\u30A6\u30A7\u30A2\u8A66\u9A13"
<!-- -->
# For amd64:
wget https://clickhouse-builds.s3.yandex.net/0/00ba767f5d2a929394ea3be193b1f79074a1c4bc/1578163263_binary/clickhouse
wget https://builds.clickhouse.com/master/amd64/clickhouse
# For aarch64:
wget https://clickhouse-builds.s3.yandex.net/0/00ba767f5d2a929394ea3be193b1f79074a1c4bc/1578161264_binary/clickhouse
wget https://builds.clickhouse.com/master/aarch64/clickhouse
# For powerpc64le:
wget https://builds.clickhouse.com/master/powerpc64le/clickhouse
# For freebsd:
wget https://builds.clickhouse.com/master/freebsd/clickhouse
# For freebsd-aarch64:
wget https://builds.clickhouse.com/master/freebsd-aarch64/clickhouse
# For freebsd-powerpc64le:
wget https://builds.clickhouse.com/master/freebsd-powerpc64le/clickhouse
# For macos:
wget https://builds.clickhouse.com/master/macos/clickhouse
# For macos-aarch64:
wget https://builds.clickhouse.com/master/macos-aarch64/clickhouse
# Then do:
chmod a+x clickhouse

View File

@ -6,6 +6,7 @@ changelog/2017.md whats-new/changelog/2017.md
changelog/2018.md whats-new/changelog/2018.md
changelog/2019.md whats-new/changelog/2019.md
changelog/index.md whats-new/changelog/index.md
commercial/cloud.md https://clickhouse.com/cloud/
data_types/array.md sql-reference/data-types/array.md
data_types/boolean.md sql-reference/data-types/boolean.md
data_types/date.md sql-reference/data-types/date.md

View File

@ -6,7 +6,7 @@ toc_title: "Навигация по коду ClickHouse"
# Навигация по коду ClickHouse {#navigatsiia-po-kodu-clickhouse}
Для навигации по коду онлайн доступен **Woboq**, он расположен [здесь](https://clickhouse.com/codebrowser/html_report///ClickHouse/src/index.html). В нём реализовано удобное перемещение между исходными файлами, семантическая подсветка, подсказки, индексация и поиск. Слепок кода обновляется ежедневно.
Для навигации по коду онлайн доступен **Woboq**, он расположен [здесь](https://clickhouse.com/codebrowser/ClickHouse/src/index.html). В нём реализовано удобное перемещение между исходными файлами, семантическая подсветка, подсказки, индексация и поиск. Слепок кода обновляется ежедневно.
Также вы можете просматривать исходники на [GitHub](https://github.com/ClickHouse/ClickHouse).

View File

@ -27,11 +27,17 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su
{% include 'install/deb.sh' %}
```
Также эти пакеты можно скачать и установить вручную отсюда: https://repo.clickhouse.com/deb/stable/main/.
<details markdown="1">
<summary>Устаревший способ установки deb-пакетов</summary>
``` bash
{% include 'install/deb_repo.sh' %}
```
</details>
Чтобы использовать различные [версии ClickHouse](../faq/operations/production.md) в зависимости от ваших потребностей, вы можете заменить `stable` на `lts` или `testing`.
Также вы можете вручную скачать и установить пакеты из [репозитория](https://repo.clickhouse.com/deb/stable/main/).
Также вы можете вручную скачать и установить пакеты из [репозитория](https://packages.clickhouse.com/deb/pool/stable).
#### Пакеты {#packages}
@ -51,11 +57,17 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su
Сначала нужно подключить официальный репозиторий:
``` bash
sudo yum install yum-utils
sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG
sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64
{% include 'install/rpm.sh' %}
```
<details markdown="1">
<summary>Устаревший способ установки rpm-пакетов</summary>
``` bash
{% include 'install/rpm_repo.sh' %}
```
</details>
Для использования наиболее свежих версий нужно заменить `stable` на `testing` (рекомендуется для тестовых окружений). Также иногда доступен `prestable`.
Для, собственно, установки пакетов необходимо выполнить следующие команды:
@ -64,36 +76,27 @@ sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64
sudo yum install clickhouse-server clickhouse-client
```
Также есть возможность установить пакеты вручную, скачав отсюда: https://repo.clickhouse.com/rpm/stable/x86_64.
Также есть возможность установить пакеты вручную, скачав отсюда: https://packages.clickhouse.com/rpm/stable.
### Из Tgz архивов {#from-tgz-archives}
Команда ClickHouse в Яндексе рекомендует использовать предкомпилированные бинарники из `tgz` архивов для всех дистрибутивов, где невозможна установка `deb` и `rpm` пакетов.
Интересующую версию архивов можно скачать вручную с помощью `curl` или `wget` из репозитория https://repo.clickhouse.com/tgz/.
Интересующую версию архивов можно скачать вручную с помощью `curl` или `wget` из репозитория https://packages.clickhouse.com/tgz/.
После этого архивы нужно распаковать и воспользоваться скриптами установки. Пример установки самой свежей версии:
``` bash
export LATEST_VERSION=`curl https://api.github.com/repos/ClickHouse/ClickHouse/tags 2>/dev/null | grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | head -n 1`
curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-dbg-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/clickhouse-server-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/clickhouse-client-$LATEST_VERSION.tgz
tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz
sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh
tar -xzvf clickhouse-common-static-dbg-$LATEST_VERSION.tgz
sudo clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh
tar -xzvf clickhouse-server-$LATEST_VERSION.tgz
sudo clickhouse-server-$LATEST_VERSION/install/doinst.sh
sudo /etc/init.d/clickhouse-server start
tar -xzvf clickhouse-client-$LATEST_VERSION.tgz
sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh
{% include 'install/tgz.sh' %}
```
<details markdown="1">
<summary>Устаревший способ установки из архивов tgz</summary>
``` bash
{% include 'install/tgz_repo.sh' %}
```
</details>
Для production окружений рекомендуется использовать последнюю `stable`-версию. Её номер также можно найти на github с на вкладке https://github.com/ClickHouse/ClickHouse/tags c постфиксом `-stable`.
### Из Docker образа {#from-docker-image}
@ -195,4 +198,4 @@ SELECT 1
**Поздравляем, система работает!**
Для дальнейших экспериментов можно попробовать загрузить один из тестовых наборов данных или пройти [пошаговое руководство для начинающих](https://clickhouse.com/tutorial.html).
Для дальнейших экспериментов можно попробовать загрузить один из тестовых наборов данных или пройти [пошаговое руководство для начинающих](./tutorial.md).

View File

@ -11,7 +11,7 @@ ClickHouse поддерживает следующий синтаксис:
- `LIMIT [offset_value, ]n BY expressions`
- `LIMIT n OFFSET offset_value BY expressions`
Во время обработки запроса, ClickHouse выбирает данные, упорядоченные по ключу сортировки. Ключ сортировки задаётся явно в секции [ORDER BY](order-by.md#select-order-by) или неявно в свойствах движка таблицы. Затем ClickHouse применяет `LIMIT n BY expressions` и возвращает первые `n` для каждой отличной комбинации `expressions`. Если указан `OFFSET`, то для каждого блока данных, который принадлежит отдельной комбинации `expressions`, ClickHouse отступает `offset_value` строк от начала блока и возвращает не более `n`. Если `offset_value` больше, чем количество строк в блоке данных, ClickHouse не возвращает ни одной строки.
Во время обработки запроса, ClickHouse выбирает данные, упорядоченные по ключу сортировки. Ключ сортировки задаётся явно в секции [ORDER BY](order-by.md#select-order-by) или неявно в свойствах движка таблицы (порядок строк гарантирован только при использовании [ORDER BY](order-by.md#select-order-by), в ином случае блоки строк не будут упорядочены из-за многопоточной обработки). Затем ClickHouse применяет `LIMIT n BY expressions` и возвращает первые `n` для каждой отличной комбинации `expressions`. Если указан `OFFSET`, то для каждого блока данных, который принадлежит отдельной комбинации `expressions`, ClickHouse отступает `offset_value` строк от начала блока и возвращает не более `n`. Если `offset_value` больше, чем количество строк в блоке данных, ClickHouse не возвращает ни одной строки.
`LIMIT BY` не связана с секцией `LIMIT`. Их можно использовать в одном запросе.

View File

@ -31,7 +31,12 @@ def build_redirect_html(args, base_prefix, lang, output_dir, from_path, to_path)
from_path.replace('/index.md', '/index.html').replace('.md', '/index.html')
)
target_path = to_path.replace('/index.md', '/').replace('.md', '/')
to_url = f'/{base_prefix}/{lang}/{target_path}'
if target_path[0:7] != 'http://' and target_path[0:8] != 'https://':
to_url = f'/{base_prefix}/{lang}/{target_path}'
else:
to_url = target_path
to_url = to_url.strip()
write_redirect_html(out_path, to_url)

View File

@ -247,7 +247,7 @@ toc_title: "\u53D8\u66F4\u65E5\u5FD7"
- 更新了clickhouse-test脚本中挂起查询的检查 [#8858](https://github.com/ClickHouse/ClickHouse/pull/8858) ([亚历山大\*卡扎科夫](https://github.com/Akazz))
- 从存储库中删除了一些无用的文件。 [#8843](https://github.com/ClickHouse/ClickHouse/pull/8843) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov))
- 更改类型的数学perftests从 `once``loop`. [#8783](https://github.com/ClickHouse/ClickHouse/pull/8783) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai))
- 添加码头镜像它允许为我们的代码库构建交互式代码浏览器HTML报告。 [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([阿利沙平](https://github.com/alesapin))见 [Woboq代码浏览器](https://clickhouse.com/codebrowser/html_report///ClickHouse/dbms/index.html)
- 添加码头镜像它允许为我们的代码库构建交互式代码浏览器HTML报告。 [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([阿利沙平](https://github.com/alesapin))见 [Woboq代码浏览器](https://clickhouse.com/codebrowser/ClickHouse/dbms/index.html)
- 抑制MSan下的一些测试失败。 [#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alexander Kuzmenkov](https://github.com/akuzm))
- 加速 “exception while insert” 测试 此测试通常在具有复盖率的调试版本中超时。 [#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov))
- 更新 `libcxx``libcxxabi` 为了主人 在准备 [#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov))

View File

@ -5,7 +5,7 @@ toc_title: "\u6D4F\u89C8\u6E90\u4EE3\u7801"
# 浏览ClickHouse源代码 {#browse-clickhouse-source-code}
您可以使用 **Woboq** 在线代码浏览器 [点击这里](https://clickhouse.com/codebrowser/html_report/ClickHouse/src/index.html). 它提供了代码导航和语义突出显示、搜索和索引。 代码快照每天更新。
您可以使用 **Woboq** 在线代码浏览器 [点击这里](https://clickhouse.com/codebrowser/ClickHouse/src/index.html). 它提供了代码导航和语义突出显示、搜索和索引。 代码快照每天更新。
此外,您还可以像往常一样浏览源代码 [GitHub](https://github.com/ClickHouse/ClickHouse)

View File

@ -38,5 +38,46 @@ CREATE TABLE test
ENGINE = EmbeddedRocksDB
PRIMARY KEY key
```
## 指标
还有一个`system.rocksdb` 表, 公开rocksdb的统计信息:
```sql
SELECT
name,
value
FROM system.rocksdb
┌─name──────────────────────┬─value─┐
│ no.file.opens │ 1 │
│ number.block.decompressed │ 1 │
└───────────────────────────┴───────┘
```
## 配置
你能修改任何[rocksdb options](https://github.com/facebook/rocksdb/wiki/Option-String-and-Option-Map) 配置,使用配置文件:
```xml
<rocksdb>
<options>
<max_background_jobs>8</max_background_jobs>
</options>
<column_family_options>
<num_levels>2</num_levels>
</column_family_options>
<tables>
<table>
<name>TABLE</name>
<options>
<max_background_jobs>8</max_background_jobs>
</options>
<column_family_options>
<num_levels>2</num_levels>
</column_family_options>
</table>
</tables>
</rocksdb>
```
[原始文章](https://clickhouse.com/docs/en/engines/table-engines/integrations/embedded-rocksdb/) <!--hide-->

View File

@ -7,7 +7,7 @@
- ReplicatedReplacingMergeTree
- ReplicatedAggregatingMergeTree
- ReplicatedCollapsingMergeTree
- ReplicatedVersionedCollapsingMergetree
- ReplicatedVersionedCollapsingMergeTree
- ReplicatedGraphiteMergeTree
副本是表级别的,不是整个服务器级的。所以,服务器里可以同时有复制表和非复制表。

View File

@ -1 +0,0 @@
../../../en/faq/integration/file-export.md

View File

@ -0,0 +1,37 @@
---
title: 如何从 ClickHouse 导出数据到一个文件?
toc_hidden: true
toc_priority: 10
---
# 如何从 ClickHouse 导出数据到一个文件? {#how-to-export-to-file}
## 使用 INTO OUTFILE 语法 {#using-into-outfile-clause}
加一个 [INTO OUTFILE](../../sql-reference/statements/select/into-outfile.md#into-outfile-clause) 语法到你的查询语句中.
例如:
``` sql
SELECT * FROM table INTO OUTFILE 'file'
```
ClickHouse 默认使用[TabSeparated](../../interfaces/formats.md#tabseparated) 格式写入数据. 修改[数据格式](../../interfaces/formats.md), 请用 [FORMAT 语法](../../sql-reference/statements/select/format.md#format-clause).
例如:
``` sql
SELECT * FROM table INTO OUTFILE 'file' FORMAT CSV
```
## 使用一个文件引擎表 {#using-a-file-engine-table}
查看 [File](../../engines/table-engines/special/file.md) 表引擎.
## 使用命令行重定向 {#using-command-line-redirection}
``` bash
$ clickhouse-client --query "SELECT * from table" --format FormatName > result.txt
```
查看 [clickhouse-client](../../interfaces/cli.md).

View File

@ -27,9 +27,17 @@ $ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not
{% include 'install/deb.sh' %}
```
<details markdown="1">
<summary>Deprecated Method for installing deb-packages</summary>
``` bash
{% include 'install/deb_repo.sh' %}
```
</details>
如果您想使用最新的版本,请用`testing`替代`stable`(我们只推荐您用于测试环境)。
你也可以从这里手动下载安装包:[下载](https://repo.clickhouse.com/deb/stable/main/)。
你也可以从这里手动下载安装包:[下载](https://packages.clickhouse.com/deb/pool/stable)。
安装包列表:
@ -45,11 +53,17 @@ $ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not
首先,您需要添加官方存储库:
``` bash
sudo yum install yum-utils
sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG
sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64
{% include 'install/rpm.sh' %}
```
<details markdown="1">
<summary>Deprecated Method for installing rpm-packages</summary>
``` bash
{% include 'install/rpm_repo.sh' %}
```
</details>
如果您想使用最新的版本,请用`testing`替代`stable`(我们只推荐您用于测试环境)。`prestable`有时也可用。
然后运行命令安装:
@ -58,37 +72,28 @@ sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64
sudo yum install clickhouse-server clickhouse-client
```
你也可以从这里手动下载安装包:[下载](https://repo.clickhouse.com/rpm/stable/x86_64)。
你也可以从这里手动下载安装包:[下载](https://packages.clickhouse.com/rpm/stable)。
### `Tgz`安装包 {#from-tgz-archives}
如果您的操作系统不支持安装`deb`或`rpm`包,建议使用官方预编译的`tgz`软件包。
所需的版本可以通过`curl`或`wget`从存储库`https://repo.clickhouse.com/tgz/`下载。
所需的版本可以通过`curl`或`wget`从存储库`https://packages.clickhouse.com/tgz/`下载。
下载后解压缩下载资源文件并使用安装脚本进行安装。以下是一个最新稳定版本的安装示例:
``` bash
export LATEST_VERSION=`curl https://api.github.com/repos/ClickHouse/ClickHouse/tags 2>/dev/null | grep stable | grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | head -n 1`
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-dbg-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-server-$LATEST_VERSION.tgz
curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-client-$LATEST_VERSION.tgz
tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz
sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh
tar -xzvf clickhouse-common-static-dbg-$LATEST_VERSION.tgz
sudo clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh
tar -xzvf clickhouse-server-$LATEST_VERSION.tgz
sudo clickhouse-server-$LATEST_VERSION/install/doinst.sh
sudo /etc/init.d/clickhouse-server start
tar -xzvf clickhouse-client-$LATEST_VERSION.tgz
sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh
{% include 'install/tgz.sh' %}
```
<details markdown="1">
<summary>Deprecated Method for installing tgz archives</summary>
``` bash
{% include 'install/tgz_repo.sh' %}
```
</details>
对于生产环境,建议使用最新的`stable`版本。你可以在GitHub页面https://github.com/ClickHouse/ClickHouse/tags找到它它以后缀`-stable`标志。
### `Docker`安装包 {#from-docker-image}
@ -183,6 +188,6 @@ SELECT 1
**恭喜,系统已经工作了!**
为了继续进行实验,你可以尝试下载测试数据集或查看[教程](https://clickhouse.com/tutorial.html)。
为了继续进行实验,你可以尝试下载测试数据集或查看[教程](./tutorial.md)。
[原始文章](https://clickhouse.com/docs/en/getting_started/install/) <!--hide-->

View File

@ -36,6 +36,18 @@ chmod a+x ./hardware.sh
wget https://builds.clickhouse.com/master/amd64/clickhouse
# For aarch64:
wget https://builds.clickhouse.com/master/aarch64/clickhouse
# For powerpc64le:
wget https://builds.clickhouse.com/master/powerpc64le/clickhouse
# For freebsd:
wget https://builds.clickhouse.com/master/freebsd/clickhouse
# For freebsd-aarch64:
wget https://builds.clickhouse.com/master/freebsd-aarch64/clickhouse
# For freebsd-powerpc64le:
wget https://builds.clickhouse.com/master/freebsd-powerpc64le/clickhouse
# For macos:
wget https://builds.clickhouse.com/master/macos/clickhouse
# For macos-aarch64:
wget https://builds.clickhouse.com/master/macos-aarch64/clickhouse
# Then do:
chmod a+x clickhouse
```

View File

@ -1 +0,0 @@
../../../../en/sql-reference/statements/alter/role.md

View File

@ -0,0 +1,16 @@
---
toc_priority: 46
toc_title: 角色
---
## 操作角色 {#alter-role-statement}
修改角色.
语法示例:
``` sql
ALTER ROLE [IF EXISTS] name1 [ON CLUSTER cluster_name1] [RENAME TO new_name1]
[, name2 [ON CLUSTER cluster_name2] [RENAME TO new_name2] ...]
[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]
```

View File

@ -1 +0,0 @@
../../../../en/sql-reference/statements/alter/row-policy.md

View File

@ -0,0 +1,19 @@
---
toc_priority: 47
toc_title: 行策略
---
# 操作行策略 {#alter-row-policy-statement}
修改行策略.
语法:
``` sql
ALTER [ROW] POLICY [IF EXISTS] name1 [ON CLUSTER cluster_name1] ON [database1.]table1 [RENAME TO new_name1]
[, name2 [ON CLUSTER cluster_name2] ON [database2.]table2 [RENAME TO new_name2] ...]
[AS {PERMISSIVE | RESTRICTIVE}]
[FOR SELECT]
[USING {condition | NONE}][,...]
[TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
```

View File

@ -11,7 +11,7 @@ ClickHouse支持以下语法变体:
- `LIMIT [offset_value, ]n BY expressions`
- `LIMIT n OFFSET offset_value BY expressions`
查询处理过程中ClickHouse会选择按排序键排序的数据。 排序键使用以下命令显式设置 [ORDER BY](../../../sql-reference/statements/select/order-by.md) 子句或隐式作为表引擎的属性。 然后ClickHouse应用 `LIMIT n BY expressions` 并返回第一 `n` 每个不同组合的行 `expressions`. 如果 `OFFSET` 被指定,则对于每个数据块属于一个不同的组合 `expressions`ClickHouse跳过 `offset_value` 从块开始的行数,并返回最大值 `n` 行的结果。 如果 `offset_value` 如果数据块中的行数大于数据块中的行数ClickHouse将从该块返回零行。
进行查询处理时ClickHouse选择按排序键排序的数据。排序键设置显式地使用一个[ORDER BY](order-by.md#select-order-by)条款或隐式属性表的引擎(行顺序只是保证在使用[ORDER BY](order-by.md#select-order-by),否则不会命令行块由于多线程)。然后ClickHouse应用`LIMIT n BY 表达式`,并为每个不同的`表达式`组合返回前n行。如果指定了`OFFSET`,那么对于每个属于不同`表达式`组合的数据块ClickHouse将跳过`offset_value`从块开始的行数,并最终返回最多`n`行的结果。如果`offset_value`大于数据块中的行数则ClickHouse从数据块中返回零行。
!!! note "注"
`LIMIT BY` 是不相关的 [LIMIT](../../../sql-reference/statements/select/limit.md). 它们都可以在同一个查询中使用。

View File

@ -2962,7 +2962,7 @@
* 更新了对 clickhouse-test 脚本中挂起查询的检查. [#8858](https://github.com/ClickHouse/ClickHouse/pull/8858) ([Alexander Kazakov](https://github.com/Akazz))
* 从存储库中删除了一些无用的文件. [#8843](https://github.com/ClickHouse/ClickHouse/pull/8843) ([alexey-milovidov](https://github.com/alexey-milovidov))
* 将数学性能测试的类型从 `once` 更改为 `loop` . [#8783](https://github.com/ClickHouse/ClickHouse/pull/8783) ([Nikolai Kochetov](https://github.com/KochetovNicolai))
* 添加 docker 图像,它允许为我们的代码库构建交互式代码浏览器 HTML 报告. [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([alesapin](https://github.com/alesapin)) See [Woboq Code Browser](https://clickhouse-test-reports.s3.yandex.net/codebrowser/html_report///ClickHouse/dbms/index.html)
* 添加 docker 图像,它允许为我们的代码库构建交互式代码浏览器 HTML 报告. [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([alesapin](https://github.com/alesapin)) See [Woboq Code Browser](https://clickhouse-test-reports.s3.yandex.net/codebrowser/ClickHouse/dbms/index.html)
* 抑制 MSan 下的一些测试失败. [#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alexander Kuzmenkov](https://github.com/akuzm))
* 加速 `exception while insert` 测试. 此测试经常在 debug-with-coverage 构建中超时. [#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([alexey-milovidov](https://github.com/alexey-milovidov))
* 将 `libcxx``libcxxabi` 更新为 master. 准备 [#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([alexey-milovidov](https://github.com/alexey-milovidov))

View File

@ -0,0 +1,31 @@
[clickhouse-stable]
name=ClickHouse - Stable Repository
baseurl=https://packages.clickhouse.com/rpm/stable/
gpgkey=https://packages.clickhouse.com/rpm/stable/repodata/repomd.xml.key
gpgcheck=0
repo_gpgcheck=1
enabled=0
[clickhouse-lts]
name=ClickHouse - LTS Repository
baseurl=https://packages.clickhouse.com/rpm/lts/
gpgkey=https://packages.clickhouse.com/rpm/lts/repodata/repomd.xml.key
gpgcheck=0
repo_gpgcheck=1
enabled=0
[clickhouse-prestable]
name=ClickHouse - Pre-stable Repository
baseurl=https://packages.clickhouse.com/rpm/prestable/
gpgkey=https://packages.clickhouse.com/rpm/prestable/repodata/repomd.xml.key
gpgcheck=0
repo_gpgcheck=1
enabled=0
[clickhouse-testing]
name=ClickHouse - Testing Repository
baseurl=https://packages.clickhouse.com/rpm/testing/
gpgkey=https://packages.clickhouse.com/rpm/testing/repodata/repomd.xml.key
gpgcheck=0
repo_gpgcheck=1
enabled=1

View File

@ -371,6 +371,13 @@ void Client::initialize(Poco::Util::Application & self)
configReadClient(config(), home_path);
const char * env_user = getenv("CLICKHOUSE_USER");
const char * env_password = getenv("CLICKHOUSE_PASSWORD");
if (env_user)
config().setString("user", env_user);
if (env_password)
config().setString("password", env_password);
// global_context->setApplicationType(Context::ApplicationType::CLIENT);
global_context->setQueryParameters(query_parameters);
@ -1119,7 +1126,12 @@ void Client::processOptions(const OptionsDescription & options_description,
{
const auto & name = setting.getName();
if (options.count(name))
config().setString(name, options[name].as<String>());
{
if (allow_repeated_settings)
config().setString(name, options[name].as<Strings>().back());
else
config().setString(name, options[name].as<String>());
}
}
if (options.count("config-file") && options.count("config"))

View File

@ -13,6 +13,7 @@
#include <base/logger_useful.h>
#include <base/ErrorHandlers.h>
#include <base/scope_guard.h>
#include <base/safeExit.h>
#include <Poco/Net/NetException.h>
#include <Poco/Net/TCPServerParams.h>
#include <Poco/Net/TCPServer.h>
@ -34,11 +35,6 @@
#include <Server/ProtocolServerAdapter.h>
#include <Server/KeeperTCPHandlerFactory.h>
#if defined(OS_LINUX)
# include <unistd.h>
# include <sys/syscall.h>
#endif
int mainEntryClickHouseKeeper(int argc, char ** argv)
{
@ -127,18 +123,6 @@ Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port
return socket_address;
}
[[noreturn]] void forceShutdown()
{
#if defined(THREAD_SANITIZER) && defined(OS_LINUX)
/// Thread sanitizer tries to do something on exit that we don't need if we want to exit immediately,
/// while connection handling threads are still run.
(void)syscall(SYS_exit_group, 0);
__builtin_unreachable();
#else
_exit(0);
#endif
}
std::string getUserName(uid_t user_id)
{
/// Try to convert user id into user name.
@ -474,7 +458,7 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
if (current_connections)
{
LOG_INFO(log, "Will shutdown forcefully.");
forceShutdown();
safeExit(0);
}
});

View File

@ -411,7 +411,8 @@ void LocalServer::setupUsers()
void LocalServer::connect()
{
connection_parameters = ConnectionParameters(config());
connection = LocalConnection::createConnection(connection_parameters, global_context, need_render_progress);
connection = LocalConnection::createConnection(
connection_parameters, global_context, need_render_progress, need_render_profile_events, server_display_name);
}

View File

@ -909,7 +909,7 @@ public:
ColumnPtr new_nested_column = nested_model->generate(nested_column);
return ColumnArray::create(IColumn::mutate(std::move(new_nested_column)), IColumn::mutate(std::move(column_array.getOffsetsPtr())));
return ColumnArray::create(IColumn::mutate(std::move(new_nested_column)), IColumn::mutate(column_array.getOffsetsPtr()));
}
void updateSeed() override
@ -947,7 +947,7 @@ public:
ColumnPtr new_nested_column = nested_model->generate(nested_column);
return ColumnNullable::create(IColumn::mutate(std::move(new_nested_column)), IColumn::mutate(std::move(column_nullable.getNullMapColumnPtr())));
return ColumnNullable::create(IColumn::mutate(std::move(new_nested_column)), IColumn::mutate(column_nullable.getNullMapColumnPtr()));
}
void updateSeed() override

View File

@ -22,6 +22,8 @@
#include <base/getMemoryAmount.h>
#include <base/errnoToString.h>
#include <base/coverage.h>
#include <base/getFQDNOrHostName.h>
#include <base/safeExit.h>
#include <Common/MemoryTracker.h>
#include <Common/ClickHouseRevision.h>
#include <Common/DNSResolver.h>
@ -31,7 +33,6 @@
#include <Common/StringUtils/StringUtils.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/ZooKeeperNodeCache.h>
#include <base/getFQDNOrHostName.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/getExecutablePath.h>
@ -95,8 +96,6 @@
# include <sys/mman.h>
# include <sys/ptrace.h>
# include <Common/hasLinuxCapability.h>
# include <unistd.h>
# include <sys/syscall.h>
#endif
#if USE_SSL
@ -505,19 +504,6 @@ void checkForUsersNotInMainConfig(
}
}
[[noreturn]] void forceShutdown()
{
#if defined(THREAD_SANITIZER) && defined(OS_LINUX)
/// Thread sanitizer tries to do something on exit that we don't need if we want to exit immediately,
/// while connection handling threads are still run.
(void)syscall(SYS_exit_group, 0);
__builtin_unreachable();
#else
_exit(0);
#endif
}
int Server::main(const std::vector<std::string> & /*args*/)
{
Poco::Logger * log = &logger();
@ -1317,7 +1303,7 @@ if (ThreadFuzzer::instance().isEffective())
#endif
#if !defined(__x86_64__)
LOG_INFO(log, "Query Profiler is only tested on x86_64. It also known to not work under qemu-user.");
LOG_INFO(log, "Query Profiler and TraceCollector is only tested on x86_64. It also known to not work under qemu-user.");
#endif
if (!hasPHDRCache())
@ -1527,7 +1513,7 @@ if (ThreadFuzzer::instance().isEffective())
/// Dump coverage here, because std::atexit callback would not be called.
dumpCoverageReportIfPossible();
LOG_INFO(log, "Will shutdown forcefully.");
forceShutdown();
safeExit(0);
}
});

View File

@ -87,13 +87,6 @@
color: var(--text-color);
}
/* Otherwise scrollbar may appear dynamically and it will alter viewport height,
then relative heights of elements will change suddenly, and it will break overall impression. */
/* html
{
overflow-x: scroll;
}*/
div
{
width: 100%;
@ -382,7 +375,7 @@
<script type="text/javascript">
/// Incremental request number. When response is received,
/// if it's request number does not equal to the current request number, response will be ignored.
/// if its request number does not equal to the current request number, response will be ignored.
/// This is to avoid race conditions.
let request_num = 0;
@ -549,7 +542,7 @@
document.getElementById('logo-container').style.display = 'block';
}
function formatReadable(number = 0, decimals = 2, units = []) {
const k = 1000;
const i = number ? Math.floor(Math.log(number) / Math.log(k)) : 0;
@ -759,20 +752,25 @@
svg.style.height = graph.graph().height;
}
function setColorTheme(theme)
{
function setColorTheme(theme) {
window.localStorage.setItem('theme', theme);
document.documentElement.setAttribute('data-theme', theme);
}
/// The choice of color theme is saved in browser.
let theme = window.localStorage.getItem('theme');
/**
* First we check if theme is set via the 'theme' GET parameter, if not, we check localStorage,
* otherwise we check OS preference
*/
let theme = current_url.searchParams.get('theme');
if (['dark', 'light'].indexOf(theme) === -1) {
theme = window.localStorage.getItem('theme');
}
if (theme) {
setColorTheme(theme);
document.documentElement.setAttribute('data-theme', theme);
} else {
/// Obtain system-level user preference
let media_query_list = window.matchMedia('prefers-color-scheme: dark')
const media_query_list = window.matchMedia('(prefers-color-scheme: dark)');
if (media_query_list.matches) {
/// Set without saving to localstorage
document.documentElement.setAttribute('data-theme', 'dark');
@ -788,13 +786,11 @@
});
}
document.getElementById('toggle-light').onclick = function()
{
document.getElementById('toggle-light').onclick = function() {
setColorTheme('light');
}
document.getElementById('toggle-dark').onclick = function()
{
document.getElementById('toggle-dark').onclick = function() {
setColorTheme('dark');
}
</script>

View File

@ -91,7 +91,7 @@ String serializeAccessEntity(const IAccessEntity & entity)
return buf.str();
}
AccessEntityPtr deserializeAccessEntity(const String & definition, const String & path)
AccessEntityPtr deserializeAccessEntityImpl(const String & definition)
{
ASTs queries;
ParserAttachAccessEntity parser;
@ -118,43 +118,42 @@ AccessEntityPtr deserializeAccessEntity(const String & definition, const String
if (auto * create_user_query = query->as<ASTCreateUserQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
throw Exception("Two access entities attached in the same file", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = user = std::make_unique<User>();
InterpreterCreateUserQuery::updateUserFromQuery(*user, *create_user_query);
}
else if (auto * create_role_query = query->as<ASTCreateRoleQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
throw Exception("Two access entities attached in the same file", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = role = std::make_unique<Role>();
InterpreterCreateRoleQuery::updateRoleFromQuery(*role, *create_role_query);
}
else if (auto * create_policy_query = query->as<ASTCreateRowPolicyQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
throw Exception("Two access entities attached in the same file", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = policy = std::make_unique<RowPolicy>();
InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(*policy, *create_policy_query);
}
else if (auto * create_quota_query = query->as<ASTCreateQuotaQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
throw Exception("Two access entities attached in the same file", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = quota = std::make_unique<Quota>();
InterpreterCreateQuotaQuery::updateQuotaFromQuery(*quota, *create_quota_query);
}
else if (auto * create_profile_query = query->as<ASTCreateSettingsProfileQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
throw Exception("Two access entities attached in the same file", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = profile = std::make_unique<SettingsProfile>();
InterpreterCreateSettingsProfileQuery::updateSettingsProfileFromQuery(*profile, *create_profile_query);
}
else if (auto * grant_query = query->as<ASTGrantQuery>())
{
if (!user && !role)
throw Exception(
"A user or role should be attached before grant in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
throw Exception("A user or role should be attached before grant", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
if (user)
InterpreterGrantQuery::updateUserFromQuery(*user, *grant_query);
else
@ -165,9 +164,27 @@ AccessEntityPtr deserializeAccessEntity(const String & definition, const String
}
if (!res)
throw Exception("No access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
throw Exception("No access entities attached", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
return res;
}
AccessEntityPtr deserializeAccessEntity(const String & definition, const String & file_path)
{
if (file_path.empty())
return deserializeAccessEntityImpl(definition);
try
{
return deserializeAccessEntityImpl(definition);
}
catch (Exception & e)
{
e.addMessage("Could not parse " + file_path);
e.rethrow();
__builtin_unreachable();
}
}
}

View File

@ -10,6 +10,6 @@ using AccessEntityPtr = std::shared_ptr<const IAccessEntity>;
String serializeAccessEntity(const IAccessEntity & entity);
AccessEntityPtr deserializeAccessEntity(const String & definition, const String & path);
AccessEntityPtr deserializeAccessEntity(const String & definition, const String & file_path = "");
}

View File

@ -706,8 +706,8 @@ private:
AccessRights::AccessRights() = default;
AccessRights::~AccessRights() = default;
AccessRights::AccessRights(AccessRights && src) = default;
AccessRights & AccessRights::operator =(AccessRights && src) = default;
AccessRights::AccessRights(AccessRights && src) noexcept = default;
AccessRights & AccessRights::operator =(AccessRights && src) noexcept = default;
AccessRights::AccessRights(const AccessRights & src)

View File

@ -19,8 +19,8 @@ public:
~AccessRights();
AccessRights(const AccessRights & src);
AccessRights & operator =(const AccessRights & src);
AccessRights(AccessRights && src);
AccessRights & operator =(AccessRights && src);
AccessRights(AccessRights && src) noexcept;
AccessRights & operator =(AccessRights && src) noexcept;
bool isEmpty() const;

View File

@ -48,7 +48,7 @@ namespace
}
catch (...)
{
tryLogCurrentException(&log, "Could not parse " + file_path);
tryLogCurrentException(&log);
return nullptr;
}
}

View File

@ -208,7 +208,7 @@ void LDAPAccessStorage::assignRolesNoLock(User & user, const LDAPClient::SearchR
{
const auto & user_name = user.getName();
auto & granted_roles = user.granted_roles;
const auto local_role_names = mapExternalRolesNoLock(external_roles);
auto local_role_names = mapExternalRolesNoLock(external_roles);
auto grant_role = [this, &user_name, &granted_roles] (const String & role_name, const bool common)
{

View File

@ -22,8 +22,8 @@ namespace ErrorCodes
RolesOrUsersSet::RolesOrUsersSet() = default;
RolesOrUsersSet::RolesOrUsersSet(const RolesOrUsersSet & src) = default;
RolesOrUsersSet & RolesOrUsersSet::operator =(const RolesOrUsersSet & src) = default;
RolesOrUsersSet::RolesOrUsersSet(RolesOrUsersSet && src) = default;
RolesOrUsersSet & RolesOrUsersSet::operator =(RolesOrUsersSet && src) = default;
RolesOrUsersSet::RolesOrUsersSet(RolesOrUsersSet && src) noexcept = default;
RolesOrUsersSet & RolesOrUsersSet::operator =(RolesOrUsersSet && src) noexcept = default;
RolesOrUsersSet::RolesOrUsersSet(AllTag)

View File

@ -22,8 +22,8 @@ struct RolesOrUsersSet
RolesOrUsersSet();
RolesOrUsersSet(const RolesOrUsersSet & src);
RolesOrUsersSet & operator =(const RolesOrUsersSet & src);
RolesOrUsersSet(RolesOrUsersSet && src);
RolesOrUsersSet & operator =(RolesOrUsersSet && src);
RolesOrUsersSet(RolesOrUsersSet && src) noexcept;
RolesOrUsersSet & operator =(RolesOrUsersSet && src) noexcept;
struct AllTag {};
RolesOrUsersSet(AllTag);

View File

@ -25,8 +25,8 @@ SettingsConstraints::SettingsConstraints(const AccessControl & access_control_)
SettingsConstraints::SettingsConstraints(const SettingsConstraints & src) = default;
SettingsConstraints & SettingsConstraints::operator=(const SettingsConstraints & src) = default;
SettingsConstraints::SettingsConstraints(SettingsConstraints && src) = default;
SettingsConstraints & SettingsConstraints::operator=(SettingsConstraints && src) = default;
SettingsConstraints::SettingsConstraints(SettingsConstraints && src) noexcept = default;
SettingsConstraints & SettingsConstraints::operator=(SettingsConstraints && src) noexcept = default;
SettingsConstraints::~SettingsConstraints() = default;

View File

@ -53,9 +53,9 @@ class SettingsConstraints
public:
SettingsConstraints(const AccessControl & access_control_);
SettingsConstraints(const SettingsConstraints & src);
SettingsConstraints & operator =(const SettingsConstraints & src);
SettingsConstraints(SettingsConstraints && src);
SettingsConstraints & operator =(SettingsConstraints && src);
SettingsConstraints & operator=(const SettingsConstraints & src);
SettingsConstraints(SettingsConstraints && src) noexcept;
SettingsConstraints & operator=(SettingsConstraints && src) noexcept;
~SettingsConstraints();
void clear();

View File

@ -147,7 +147,7 @@ void BackupImpl::readBackupMetadata()
auto in = readFileImpl(".backup");
String str;
readStringUntilEOF(str, *in);
std::istringstream stream(std::move(str)); // STYLE_CHECK_ALLOW_STD_STRING_STREAM
std::istringstream stream(str); // STYLE_CHECK_ALLOW_STD_STRING_STREAM
Poco::AutoPtr<Poco::Util::XMLConfiguration> config{new Poco::Util::XMLConfiguration()};
config->load(stream);

View File

@ -7,12 +7,13 @@
#include <map>
#include <unordered_map>
#include <base/argsToConfig.h>
#include <Common/DateLUT.h>
#include <Common/LocalDate.h>
#include <Common/MemoryTracker.h>
#include <base/argsToConfig.h>
#include <base/LineReader.h>
#include <base/scope_guard_safe.h>
#include <base/safeExit.h>
#include <Common/Exception.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/tests/gtest_global_context.h>
@ -229,11 +230,11 @@ public:
static bool cancelled() { return exit_on_signal.test(); }
};
/// This signal handler is set only for sigint.
/// This signal handler is set only for SIGINT.
void interruptSignalHandler(int signum)
{
if (exit_on_signal.test_and_set())
_exit(signum);
safeExit(128 + signum);
}
@ -243,22 +244,22 @@ ClientBase::ClientBase() = default;
void ClientBase::setupSignalHandler()
{
exit_on_signal.test_and_set();
exit_on_signal.test_and_set();
struct sigaction new_act;
memset(&new_act, 0, sizeof(new_act));
struct sigaction new_act;
memset(&new_act, 0, sizeof(new_act));
new_act.sa_handler = interruptSignalHandler;
new_act.sa_flags = 0;
new_act.sa_handler = interruptSignalHandler;
new_act.sa_flags = 0;
#if defined(OS_DARWIN)
sigemptyset(&new_act.sa_mask);
#else
if (sigemptyset(&new_act.sa_mask))
if (sigemptyset(&new_act.sa_mask))
throw Exception(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler.");
#endif
if (sigaction(SIGINT, &new_act, nullptr))
if (sigaction(SIGINT, &new_act, nullptr))
throw Exception(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler.");
}
@ -702,7 +703,6 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa
/// Also checks if query execution should be cancelled.
void ClientBase::receiveResult(ASTPtr parsed_query)
{
bool cancelled = false;
QueryInterruptHandler query_interrupt_handler;
// TODO: get the poll_interval from commandline.
@ -773,7 +773,7 @@ void ClientBase::receiveResult(ASTPtr parsed_query)
/// Receive a part of the result, or progress info or an exception and process it.
/// Returns true if one should continue receiving packets.
/// Output of result is suppressed if query was cancelled.
bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled)
bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_)
{
Packet packet = connection->receivePacket();
@ -783,7 +783,7 @@ bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled)
return true;
case Protocol::Server::Data:
if (!cancelled)
if (!cancelled_)
onData(packet.block, parsed_query);
return true;
@ -796,12 +796,12 @@ bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled)
return true;
case Protocol::Server::Totals:
if (!cancelled)
if (!cancelled_)
onTotals(packet.block, parsed_query);
return true;
case Protocol::Server::Extremes:
if (!cancelled)
if (!cancelled_)
onExtremes(packet.block, parsed_query);
return true;
@ -867,7 +867,7 @@ void ClientBase::onProfileEvents(Block & block)
if (rows == 0)
return;
if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS)
if (getName() == "local" || server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS)
{
const auto & array_thread_id = typeid_cast<const ColumnUInt64 &>(*block.getByName("thread_id").column).getData();
const auto & names = typeid_cast<const ColumnString &>(*block.getByName("name").column);
@ -1265,6 +1265,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
{
resetOutput();
have_error = false;
cancelled = false;
client_exception.reset();
server_exception.reset();
@ -1392,6 +1393,9 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
String & query_to_execute, ASTPtr & parsed_query, const String & all_queries_text,
std::optional<Exception> & current_exception)
{
if (!is_interactive && cancelled)
return MultiQueryProcessingStage::QUERIES_END;
if (this_query_begin >= all_queries_end)
return MultiQueryProcessingStage::QUERIES_END;
@ -1868,6 +1872,8 @@ void ClientBase::readArguments(
prev_port_arg = port_arg;
}
}
else if (arg == "--allow_repeated_settings"sv)
allow_repeated_settings = true;
else
common_arguments.emplace_back(arg);
}
@ -1880,7 +1886,10 @@ void ClientBase::readArguments(
void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments)
{
cmd_settings.addProgramOptions(options_description.main_description.value());
if (allow_repeated_settings)
cmd_settings.addProgramOptionsAsMultitokens(options_description.main_description.value());
else
cmd_settings.addProgramOptions(options_description.main_description.value());
/// Parse main commandline options.
auto parser = po::command_line_parser(arguments).options(options_description.main_description.value()).allow_unregistered();
po::parsed_options parsed = parser.run();

View File

@ -108,7 +108,7 @@ protected:
private:
void receiveResult(ASTPtr parsed_query);
bool receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled);
bool receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_);
void receiveLogs(ASTPtr parsed_query);
bool receiveSampleBlock(Block & out, ColumnsDescription & columns_description, ASTPtr parsed_query);
bool receiveEndOfQuery();
@ -219,6 +219,7 @@ protected:
ProgressIndication progress_indication;
bool need_render_progress = true;
bool need_render_profile_events = true;
bool written_first_block = false;
size_t processed_rows = 0; /// How many rows have been read or written.
@ -259,6 +260,10 @@ protected:
};
std::vector<HostAndPort> hosts_and_ports{};
bool allow_repeated_settings = false;
bool cancelled = false;
};
}

View File

@ -165,7 +165,7 @@ std::variant<int, ConnectionEstablisher::TryResult> ConnectionEstablisherAsync::
fiber = std::move(fiber).resume();
if (exception)
std::rethrow_exception(std::move(exception));
std::rethrow_exception(exception);
if (connection_establisher.isFinished())
{

View File

@ -363,7 +363,7 @@ bool HedgedConnections::resumePacketReceiver(const HedgedConnections::ReplicaLoc
else if (std::holds_alternative<std::exception_ptr>(res))
{
finishProcessReplica(replica_state, true);
std::rethrow_exception(std::move(std::get<std::exception_ptr>(res)));
std::rethrow_exception(std::get<std::exception_ptr>(res));
}
return false;

View File

@ -6,6 +6,8 @@
#include <Processors/Executors/PushingAsyncPipelineExecutor.h>
#include <Storages/IStorage.h>
#include <Core/Protocol.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
namespace DB
@ -18,10 +20,12 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
}
LocalConnection::LocalConnection(ContextPtr context_, bool send_progress_)
LocalConnection::LocalConnection(ContextPtr context_, bool send_progress_, bool send_profile_events_, const String & server_display_name_)
: WithContext(context_)
, session(getContext(), ClientInfo::Interface::LOCAL)
, send_progress(send_progress_)
, send_profile_events(send_profile_events_)
, server_display_name(server_display_name_)
{
/// Authenticate and create a context to execute queries.
session.authenticate("default", "", Poco::Net::SocketAddress{});
@ -58,6 +62,11 @@ void LocalConnection::updateProgress(const Progress & value)
state->progress.incrementPiecewiseAtomically(value);
}
void LocalConnection::getProfileEvents(Block & block)
{
ProfileEvents::getProfileEvents(server_display_name, state->profile_queue, block, last_sent_snapshots);
}
void LocalConnection::sendQuery(
const ConnectionTimeouts &,
const String & query,
@ -77,18 +86,23 @@ void LocalConnection::sendQuery(
if (!current_database.empty())
query_context->setCurrentDatabase(current_database);
CurrentThread::QueryScope query_scope_holder(query_context);
state.reset();
state.emplace();
state->query_id = query_id;
state->query = query;
state->query_scope_holder = std::make_unique<CurrentThread::QueryScope>(query_context);
state->stage = QueryProcessingStage::Enum(stage);
state->profile_queue = std::make_shared<InternalProfileEventsQueue>(std::numeric_limits<int>::max());
CurrentThread::attachInternalProfileEventsQueue(state->profile_queue);
if (send_progress)
state->after_send_progress.restart();
if (send_profile_events)
state->after_send_profile_events.restart();
next_packet_type.reset();
try
@ -161,11 +175,11 @@ void LocalConnection::sendData(const Block & block, const String &, bool)
if (state->pushing_async_executor)
{
state->pushing_async_executor->push(std::move(block));
state->pushing_async_executor->push(block);
}
else if (state->pushing_executor)
{
state->pushing_executor->push(std::move(block));
state->pushing_executor->push(block);
}
}
@ -231,6 +245,16 @@ bool LocalConnection::poll(size_t)
return true;
}
if (send_profile_events && (state->after_send_profile_events.elapsedMicroseconds() >= query_context->getSettingsRef().interactive_delay))
{
Block block;
state->after_send_profile_events.restart();
next_packet_type = Protocol::Server::ProfileEvents;
getProfileEvents(block);
state->block.emplace(std::move(block));
return true;
}
try
{
pollImpl();
@ -459,9 +483,14 @@ void LocalConnection::sendMergeTreeReadTaskResponse(const PartitionReadResponse
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented");
}
ServerConnectionPtr LocalConnection::createConnection(const ConnectionParameters &, ContextPtr current_context, bool send_progress)
ServerConnectionPtr LocalConnection::createConnection(
const ConnectionParameters &,
ContextPtr current_context,
bool send_progress,
bool send_profile_events,
const String & server_display_name)
{
return std::make_unique<LocalConnection>(current_context, send_progress);
return std::make_unique<LocalConnection>(current_context, send_progress, send_profile_events, server_display_name);
}

View File

@ -5,6 +5,7 @@
#include <QueryPipeline/BlockIO.h>
#include <IO/TimeoutSetter.h>
#include <Interpreters/Session.h>
#include <Interpreters/ProfileEventsExt.h>
#include <Storages/ColumnsDescription.h>
@ -29,6 +30,7 @@ struct LocalQueryState
std::unique_ptr<PullingAsyncPipelineExecutor> executor;
std::unique_ptr<PushingPipelineExecutor> pushing_executor;
std::unique_ptr<PushingAsyncPipelineExecutor> pushing_async_executor;
InternalProfileEventsQueuePtr profile_queue;
std::optional<Exception> exception;
@ -50,19 +52,28 @@ struct LocalQueryState
Progress progress;
/// Time after the last check to stop the request and send the progress.
Stopwatch after_send_progress;
Stopwatch after_send_profile_events;
std::unique_ptr<CurrentThread::QueryScope> query_scope_holder;
};
class LocalConnection : public IServerConnection, WithContext
{
public:
explicit LocalConnection(ContextPtr context_, bool send_progress_ = false);
explicit LocalConnection(
ContextPtr context_, bool send_progress_ = false, bool send_profile_events_ = false, const String & server_display_name_ = "");
~LocalConnection() override;
IServerConnection::Type getConnectionType() const override { return IServerConnection::Type::LOCAL; }
static ServerConnectionPtr createConnection(const ConnectionParameters & connection_parameters, ContextPtr current_context, bool send_progress = false);
static ServerConnectionPtr createConnection(
const ConnectionParameters & connection_parameters,
ContextPtr current_context,
bool send_progress = false,
bool send_profile_events = false,
const String & server_display_name = "");
void setDefaultDatabase(const String & database) override;
@ -129,12 +140,16 @@ private:
void updateProgress(const Progress & value);
void getProfileEvents(Block & block);
bool pollImpl();
ContextMutablePtr query_context;
Session session;
bool send_progress;
bool send_profile_events;
String server_display_name;
String description = "clickhouse-local";
std::optional<LocalQueryState> state;
@ -144,5 +159,7 @@ private:
std::optional<UInt64> next_packet_type;
String current_database;
ProfileEvents::ThreadIdToCountersSnapshot last_sent_snapshots;
};
}

View File

@ -103,6 +103,7 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p
{
loading_thread = std::thread([context=Context::createCopy(context), connection_parameters, suggestion_limit, this]
{
ThreadStatus thread_status;
for (size_t retry = 0; retry < 10; ++retry)
{
try

View File

@ -36,8 +36,8 @@ public:
static Ptr create(const ColumnPtr & column) { return ColumnMap::create(column->assumeMutable()); }
static Ptr create(ColumnPtr && arg) { return create(arg); }
template <typename Arg, typename = typename std::enable_if<std::is_rvalue_reference<Arg &&>::value>::type>
static MutablePtr create(Arg && arg) { return Base::create(std::forward<Arg>(arg)); }
template <typename ... Args, typename = typename std::enable_if<IsMutableColumns<Args ...>::value>::type>
static MutablePtr create(Args &&... args) { return Base::create(std::forward<Args>(args)...); }
std::string getName() const override;
const char * getFamilyName() const override { return "Map"; }

View File

@ -541,7 +541,7 @@ ColumnPtr ColumnNullable::compress() const
size_t byte_size = nested_column->byteSize() + null_map->byteSize();
return ColumnCompressed::create(size(), byte_size,
[nested_column = std::move(nested_column), null_map = std::move(null_map)]
[nested_column = std::move(nested_compressed), null_map = std::move(null_map_compressed)]
{
return ColumnNullable::create(nested_column->decompress(), null_map->decompress());
});

View File

@ -330,7 +330,7 @@ ColumnPtr ColumnSparse::filter(const Filter & filt, ssize_t) const
}
auto res_values = values->filter(values_filter, values_result_size_hint);
return this->create(std::move(res_values), std::move(res_offsets), res_offset);
return this->create(res_values, std::move(res_offsets), res_offset);
}
void ColumnSparse::expand(const Filter & mask, bool inverted)

View File

@ -11,12 +11,12 @@ ActionLock::ActionLock(const ActionBlocker & blocker) : counter_ptr(blocker.coun
++(*counter);
}
ActionLock::ActionLock(ActionLock && other)
ActionLock::ActionLock(ActionLock && other) noexcept
{
*this = std::move(other);
}
ActionLock & ActionLock::operator=(ActionLock && other)
ActionLock & ActionLock::operator=(ActionLock && other) noexcept
{
auto lock_lhs = this->counter_ptr.lock();

View File

@ -19,8 +19,8 @@ public:
explicit ActionLock(const ActionBlocker & blocker);
ActionLock(ActionLock && other);
ActionLock & operator=(ActionLock && other);
ActionLock(ActionLock && other) noexcept;
ActionLock & operator=(ActionLock && other) noexcept;
ActionLock(const ActionLock & other) = delete;
ActionLock & operator=(const ActionLock & other) = delete;

View File

@ -100,6 +100,11 @@ public:
return emplace(x);
}
[[nodiscard]] bool push(T && x)
{
return emplace(std::move(x));
}
/// Returns false if queue is finished
template <typename... Args>
[[nodiscard]] bool emplace(Args &&... args)
@ -120,6 +125,11 @@ public:
return emplaceImpl(milliseconds, x);
}
[[nodiscard]] bool tryPush(T && x, UInt64 milliseconds = 0)
{
return emplaceImpl(milliseconds, std::move(x));
}
/// Returns false if queue is finished or object was not emplaced during timeout
template <typename... Args>
[[nodiscard]] bool tryEmplace(UInt64 milliseconds, Args &&... args)

View File

@ -664,6 +664,10 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config,
new_path.erase(0, main_config_path.size());
std::replace(new_path.begin(), new_path.end(), '/', '_');
/// If we have config file in YAML format, the preprocessed config will inherit .yaml extension
/// but will contain config in XML format, so some tools like clickhouse extract-from-config won't work
new_path = fs::path(new_path).replace_extension(".xml").string();
if (preprocessed_dir.empty())
{
if (!loaded_config.configuration->has("path"))

View File

@ -1001,8 +1001,12 @@ public:
inline LUTIndex makeLUTIndex(Int16 year, UInt8 month, UInt8 day_of_month) const
{
if (unlikely(year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31))
if (unlikely(year < DATE_LUT_MIN_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31))
return LUTIndex(0);
if (unlikely(year > DATE_LUT_MAX_YEAR))
return LUTIndex(DATE_LUT_SIZE - 1);
auto year_lut_index = (year - DATE_LUT_MIN_YEAR) * 12 + month - 1;
UInt32 index = years_months_lut[year_lut_index].toUnderType() + day_of_month - 1;
/// When date is out of range, default value is DATE_LUT_SIZE - 1 (2283-11-11)
@ -1012,7 +1016,7 @@ public:
/// Create DayNum from year, month, day of month.
inline ExtendedDayNum makeDayNum(Int16 year, UInt8 month, UInt8 day_of_month, Int32 default_error_day_num = 0) const
{
if (unlikely(year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31))
if (unlikely(year < DATE_LUT_MIN_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31))
return ExtendedDayNum(default_error_day_num);
return toDayNum(makeLUTIndex(year, month, day_of_month));

View File

@ -21,12 +21,12 @@ Epoll::Epoll() : events_count(0)
throwFromErrno("Cannot open epoll descriptor", DB::ErrorCodes::EPOLL_ERROR);
}
Epoll::Epoll(Epoll && other) : epoll_fd(other.epoll_fd), events_count(other.events_count.load())
Epoll::Epoll(Epoll && other) noexcept : epoll_fd(other.epoll_fd), events_count(other.events_count.load())
{
other.epoll_fd = -1;
}
Epoll & Epoll::operator=(Epoll && other)
Epoll & Epoll::operator=(Epoll && other) noexcept
{
epoll_fd = other.epoll_fd;
other.epoll_fd = -1;

View File

@ -19,8 +19,8 @@ public:
Epoll(const Epoll &) = delete;
Epoll & operator=(const Epoll &) = delete;
Epoll & operator=(Epoll && other);
Epoll(Epoll && other);
Epoll & operator=(Epoll && other) noexcept;
Epoll(Epoll && other) noexcept;
/// Add new file descriptor to epoll. If ptr set to nullptr, epoll_event.data.fd = fd,
/// otherwise epoll_event.data.ptr = ptr.

View File

@ -1,7 +1,11 @@
#if defined(OS_LINUX)
#if defined(OS_LINUX) || defined(OS_FREEBSD)
#include <sys/types.h>
#include <sys/stat.h>
#if defined(OS_FREEBSD)
#include <sys/sysctl.h>
#include <sys/user.h>
#endif
#include <fcntl.h>
#include <unistd.h>
#include <cassert>
@ -18,6 +22,8 @@
namespace DB
{
#if defined(OS_LINUX)
namespace ErrorCodes
{
extern const int FILE_DOESNT_EXIST;
@ -103,6 +109,53 @@ MemoryStatisticsOS::Data MemoryStatisticsOS::get() const
return data;
}
#endif
#if defined(OS_FREEBSD)
namespace ErrorCodes
{
extern const int SYSTEM_ERROR;
}
MemoryStatisticsOS::MemoryStatisticsOS()
{
pagesize = static_cast<size_t>(::getPageSize());
self = ::getpid();
}
MemoryStatisticsOS::~MemoryStatisticsOS()
{
}
MemoryStatisticsOS::Data MemoryStatisticsOS::get() const
{
Data data;
int mib[4] = { CTL_KERN, KERN_PROC, KERN_PROC_PID, self };
struct kinfo_proc kp;
size_t len = sizeof(struct kinfo_proc);
if (-1 == ::sysctl(mib, 4, &kp, &len, NULL, 0))
throwFromErrno("Cannot sysctl(kern.proc.pid." + std::to_string(self) + ")", ErrorCodes::SYSTEM_ERROR);
if (sizeof(struct kinfo_proc) != len)
throw DB::Exception(DB::ErrorCodes::SYSTEM_ERROR, "Kernel returns structure of {} bytes instead of expected {}",
len, sizeof(struct kinfo_proc));
if (sizeof(struct kinfo_proc) != kp.ki_structsize)
throw DB::Exception(DB::ErrorCodes::SYSTEM_ERROR, "Kernel structure size ({}) does not match expected ({}).",
kp.ki_structsize, sizeof(struct kinfo_proc));
data.virt = kp.ki_size;
data.resident = kp.ki_rssize * pagesize;
data.code = kp.ki_tsize * pagesize;
data.data_and_stack = (kp.ki_dsize + kp.ki_ssize) * pagesize;
return data;
}
#endif
}
#endif

View File

@ -1,6 +1,9 @@
#pragma once
#if defined(OS_LINUX)
#if defined(OS_LINUX) || defined(OS_FREEBSD)
#include <cstdint>
#if defined(OS_FREEBSD)
#include <unistd.h>
#endif
namespace DB
@ -23,7 +26,9 @@ public:
{
uint64_t virt;
uint64_t resident;
#if defined(OS_LINUX)
uint64_t shared;
#endif
uint64_t code;
uint64_t data_and_stack;
};
@ -35,7 +40,13 @@ public:
Data get() const;
private:
#if defined(OS_LINUX)
int fd;
#endif
#if defined(OS_FREEBSD)
int pagesize;
pid_t self;
#endif
};
}

View File

@ -27,7 +27,7 @@ TimerDescriptor::TimerDescriptor(int clockid, int flags)
throwFromErrno("Cannot set O_NONBLOCK for timer_fd", ErrorCodes::CANNOT_FCNTL);
}
TimerDescriptor::TimerDescriptor(TimerDescriptor && other) : timer_fd(other.timer_fd)
TimerDescriptor::TimerDescriptor(TimerDescriptor && other) noexcept : timer_fd(other.timer_fd)
{
other.timer_fd = -1;
}

View File

@ -17,7 +17,7 @@ public:
TimerDescriptor(const TimerDescriptor &) = delete;
TimerDescriptor & operator=(const TimerDescriptor &) = delete;
TimerDescriptor(TimerDescriptor && other);
TimerDescriptor(TimerDescriptor && other) noexcept;
TimerDescriptor & operator=(TimerDescriptor &&) = default;
int getDescriptor() const { return timer_fd; }

View File

@ -59,7 +59,7 @@ static void addRootPath(String & path, const String & root_path)
throw Exception("Path cannot be empty", Error::ZBADARGUMENTS);
if (path[0] != '/')
throw Exception("Path must begin with /, got " + path, Error::ZBADARGUMENTS);
throw Exception("Path must begin with /, got path '" + path + "'", Error::ZBADARGUMENTS);
if (root_path.empty())
return;

View File

@ -52,7 +52,7 @@ private:
{
std::cerr << "Mutating\n";
auto res = shallowMutate();
res->wrapped = IColumn::mutate(std::move(wrapped));
res->wrapped = IColumn::mutate(wrapped);
return res;
}

View File

@ -79,8 +79,12 @@ FailuresCount countFailures(const ::testing::TestResult & test_result)
TEST(DateLUTTest, makeDayNumTest)
{
const DateLUTImpl & lut = DateLUT::instance("UTC");
EXPECT_EQ(0, lut.makeDayNum(2500, 12, 25));
EXPECT_EQ(0, lut.makeDayNum(1924, 12, 31));
EXPECT_EQ(-1, lut.makeDayNum(1924, 12, 31, -1));
EXPECT_EQ(-16436, lut.makeDayNum(1925, 1, 1));
EXPECT_EQ(0, lut.makeDayNum(1970, 1, 1));
EXPECT_EQ(114635, lut.makeDayNum(2283, 11, 11));
EXPECT_EQ(114635, lut.makeDayNum(2500, 12, 25));
}

View File

@ -316,7 +316,7 @@ CodecTestSequence operator+(CodecTestSequence && left, const CodecTestSequence &
std::vector<CodecTestSequence> operator+(const std::vector<CodecTestSequence> & left, const std::vector<CodecTestSequence> & right)
{
std::vector<CodecTestSequence> result(std::move(left));
std::vector<CodecTestSequence> result(left);
std::move(std::begin(right), std::end(right), std::back_inserter(result));
return result;

View File

@ -732,7 +732,7 @@ void Changelog::cleanLogThread()
while (!log_files_to_delete_queue.isFinishedAndEmpty())
{
std::string path;
if (log_files_to_delete_queue.tryPop(path))
if (log_files_to_delete_queue.pop(path))
{
std::error_code ec;
if (std::filesystem::remove(path, ec))

View File

@ -202,7 +202,7 @@ void print(IFourLetterCommand::StringBuffer & buf, const String & key, uint64_t
String MonitorCommand::run()
{
KeeperConnectionStats stats = keeper_dispatcher.getKeeperConnectionStats();
auto & stats = keeper_dispatcher.getKeeperConnectionStats();
Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo();
if (!keeper_info.has_leader)
@ -288,7 +288,7 @@ String ServerStatCommand::run()
writeText('\n', buf);
};
KeeperConnectionStats stats = keeper_dispatcher.getKeeperConnectionStats();
auto & stats = keeper_dispatcher.getKeeperConnectionStats();
Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo();
write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH);
@ -314,7 +314,7 @@ String StatCommand::run()
auto write = [&buf] (const String & key, const String & value) { buf << key << ": " << value << '\n'; };
KeeperConnectionStats stats = keeper_dispatcher.getKeeperConnectionStats();
auto & stats = keeper_dispatcher.getKeeperConnectionStats();
Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo();
write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH);

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