mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge branch 'master' into fix-links
This commit is contained in:
commit
6379cb41ce
@ -15,7 +15,7 @@ then
|
||||
# If the system has >=ARMv8.2 (https://en.wikipedia.org/wiki/AArch64), choose the corresponding build, else fall back to a v8.0
|
||||
# compat build. Unfortunately, the ARM ISA level cannot be read directly, we need to guess from the "features" in /proc/cpuinfo.
|
||||
# Also, the flags in /proc/cpuinfo are named differently than the flags passed to the compiler (cmake/cpu_features.cmake).
|
||||
ARMV82=$(grep -m 1 'Features' /proc/cpuinfo | awk '/asimd/ && /sha1/ && /aes/ && /atomics/')
|
||||
ARMV82=$(grep -m 1 'Features' /proc/cpuinfo | awk '/asimd/ && /sha1/ && /aes/ && /atomics/ && /lrcpc/')
|
||||
if [ "${ARMV82}" ]
|
||||
then
|
||||
DIR="aarch64"
|
||||
|
@ -1818,11 +1818,6 @@ Result:
|
||||
└──────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## modelEvaluate(model_name, …)
|
||||
|
||||
Evaluate external model.
|
||||
Accepts a model name and model arguments. Returns Float64.
|
||||
|
||||
## catboostEvaluate(path_to_model, feature_1, feature_2, …, feature_n)
|
||||
|
||||
Evaluate external catboost model. [CatBoost](https://catboost.ai) is an open-source gradient boosting library developed by Yandex for machine learing.
|
||||
|
@ -1722,12 +1722,6 @@ SELECT joinGet(db_test.id_val,'val',toUInt32(number)) from numbers(4) SETTINGS j
|
||||
└──────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## modelEvaluate(model_name, …) {#function-modelevaluate}
|
||||
|
||||
Оценивает внешнюю модель.
|
||||
|
||||
Принимает на вход имя и аргументы модели. Возвращает Float64.
|
||||
|
||||
## throwIf(x\[, message\[, error_code\]\]) {#throwifx-custom-message}
|
||||
|
||||
Бросает исключение, если аргумент не равен нулю.
|
||||
|
@ -625,11 +625,6 @@ ORDER BY k ASC
|
||||
|
||||
使用指定的连接键从Join类型引擎的表中获取数据。
|
||||
|
||||
## modelEvaluate(model_name, …) {#function-modelevaluate}
|
||||
|
||||
使用外部模型计算。
|
||||
接受模型的名称以及模型的参数。返回Float64类型的值。
|
||||
|
||||
## throwIf(x) {#throwifx}
|
||||
|
||||
如果参数不为零则抛出异常。
|
||||
|
@ -842,6 +842,7 @@ void Client::addOptions(OptionsDescription & options_description)
|
||||
|
||||
("no-warnings", "disable warnings when client connects to server")
|
||||
("fake-drop", "Ignore all DROP queries, should be used only for testing")
|
||||
("accept-invalid-certificate", "Ignore certificate verification errors, equal to config parameters openSSL.client.invalidCertificateHandler.name=AcceptCertificateHandler and openSSL.client.verificationMode=none")
|
||||
;
|
||||
|
||||
/// Commandline options related to external tables.
|
||||
@ -976,6 +977,13 @@ void Client::processOptions(const OptionsDescription & options_description,
|
||||
config().setBool("no-warnings", true);
|
||||
if (options.count("fake-drop"))
|
||||
fake_drop = true;
|
||||
if (options.count("accept-invalid-certificate"))
|
||||
{
|
||||
config().setString("openSSL.client.invalidCertificateHandler.name", "AcceptCertificateHandler");
|
||||
config().setString("openSSL.client.verificationMode", "none");
|
||||
}
|
||||
else
|
||||
config().setString("openSSL.client.invalidCertificateHandler.name", "RejectCertificateHandler");
|
||||
|
||||
if ((query_fuzzer_runs = options["query-fuzzer-runs"].as<int>()))
|
||||
{
|
||||
|
@ -70,6 +70,12 @@ public:
|
||||
if (!pos)
|
||||
return false;
|
||||
|
||||
/// It is possible that tables list is empty.
|
||||
/// IdentifierSemantic get the position from AST, and it can be not valid to use it.
|
||||
/// Example is re-analysing a part of AST for storage Merge, see 02147_order_by_optimizations.sql
|
||||
if (*pos >= tables.size())
|
||||
return false;
|
||||
|
||||
if (auto data_type_and_name = tables[*pos].columns.tryGetByName(identifier->shortName()))
|
||||
{
|
||||
arg_data_type = data_type_and_name->type;
|
||||
|
@ -6144,6 +6144,9 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & sour
|
||||
if (format_version != src_data->format_version)
|
||||
throw Exception("Tables have different format_version", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (query_to_string(my_snapshot->getPrimaryKeyAST()) != query_to_string(src_snapshot->getPrimaryKeyAST()))
|
||||
throw Exception("Tables have different primary key", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
return *src_data;
|
||||
}
|
||||
|
||||
|
@ -1408,31 +1408,23 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd
|
||||
ops = std::move(new_ops);
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
Coordination::Responses responses;
|
||||
Coordination::Error e = zookeeper->tryMulti(ops, responses);
|
||||
if (e == Coordination::Error::ZOK)
|
||||
return transaction.commit();
|
||||
Coordination::Responses responses;
|
||||
Coordination::Error e = zookeeper->tryMulti(ops, responses);
|
||||
if (e == Coordination::Error::ZOK)
|
||||
return transaction.commit();
|
||||
|
||||
if (e == Coordination::Error::ZNODEEXISTS)
|
||||
if (e == Coordination::Error::ZNODEEXISTS)
|
||||
{
|
||||
size_t num_check_ops = 2 * absent_part_paths_on_replicas.size();
|
||||
size_t failed_op_index = zkutil::getFailedOpIndex(e, responses);
|
||||
if (failed_op_index < num_check_ops)
|
||||
{
|
||||
size_t num_check_ops = 2 * absent_part_paths_on_replicas.size();
|
||||
size_t failed_op_index = zkutil::getFailedOpIndex(e, responses);
|
||||
if (failed_op_index < num_check_ops)
|
||||
{
|
||||
LOG_INFO(log, "The part {} on a replica suddenly appeared, will recheck checksums", ops[failed_op_index]->getPath());
|
||||
continue;
|
||||
}
|
||||
LOG_INFO(log, "The part {} on a replica suddenly appeared, will recheck checksums", ops[failed_op_index]->getPath());
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
throw zkutil::KeeperException(e);
|
||||
}
|
||||
catch (const std::exception &)
|
||||
{
|
||||
unlockSharedData(*part);
|
||||
throw;
|
||||
}
|
||||
throw zkutil::KeeperException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@ -8153,7 +8145,6 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP
|
||||
}
|
||||
catch (const Exception & ex)
|
||||
{
|
||||
unlockSharedData(*new_data_part);
|
||||
LOG_WARNING(log, "Cannot commit empty part {} with error {}", lost_part_name, ex.displayText());
|
||||
return false;
|
||||
}
|
||||
|
@ -24,6 +24,11 @@ runner_arch() {
|
||||
esac
|
||||
}
|
||||
|
||||
# We have test for cgroups, and it's broken with cgroups v2
|
||||
# Ubuntu 22.04 has it enabled by default
|
||||
sed -r '/GRUB_CMDLINE_LINUX=/ s/"(.*)"/"\1 systemd.unified_cgroup_hierarchy=0"/' -i /etc/default/grub
|
||||
update-grub
|
||||
|
||||
apt-get update
|
||||
|
||||
apt-get install --yes --no-install-recommends \
|
||||
|
@ -7,7 +7,8 @@
|
||||
:main jepsen.clickhouse-keeper.main
|
||||
:plugins [[lein-cljfmt "0.7.0"]]
|
||||
:dependencies [[org.clojure/clojure "1.10.1"]
|
||||
[jepsen "0.2.6"]
|
||||
[jepsen "0.2.7"]
|
||||
[zookeeper-clj "0.9.4"]
|
||||
[com.hierynomus/sshj "0.34.0"]
|
||||
[org.apache.zookeeper/zookeeper "3.6.1" :exclusions [org.slf4j/slf4j-log4j12]]]
|
||||
:repl-options {:init-ns jepsen.clickhouse-keeper.main})
|
||||
|
@ -1,3 +1,98 @@
|
||||
(ns jepsen.control.sshj
|
||||
(:require [jepsen.control [core :as core]
|
||||
[sshj :as sshj]]
|
||||
[slingshot.slingshot :refer [try+ throw+]])
|
||||
(:import (net.schmizz.sshj SSHClient
|
||||
DefaultConfig)
|
||||
(net.schmizz.sshj.transport.verification PromiscuousVerifier)
|
||||
(java.util.concurrent Semaphore)))
|
||||
|
||||
(defrecord SSHJRemote [concurrency-limit
|
||||
conn-spec
|
||||
^SSHClient client
|
||||
^Semaphore semaphore]
|
||||
core/Remote
|
||||
(connect [this conn-spec]
|
||||
(if (:dummy conn-spec)
|
||||
(assoc this :conn-spec conn-spec)
|
||||
(try+ (let [c (as-> (SSHClient.) client
|
||||
(do
|
||||
(if (:strict-host-key-checking conn-spec)
|
||||
(.loadKnownHosts client)
|
||||
(.addHostKeyVerifier client (PromiscuousVerifier.)))
|
||||
(.connect client (:host conn-spec) (:port conn-spec))
|
||||
(auth! client conn-spec)
|
||||
client))]
|
||||
(assoc this
|
||||
:conn-spec conn-spec
|
||||
:client c
|
||||
:semaphore (Semaphore. concurrency-limit true)))
|
||||
(catch Exception e
|
||||
; SSHJ wraps InterruptedException in its own exceptions, so we
|
||||
; have to see through that and rethrow properly.
|
||||
(let [cause (util/ex-root-cause e)]
|
||||
(when (instance? InterruptedException cause)
|
||||
(throw cause)))
|
||||
(throw+ (assoc conn-spec
|
||||
:type :jepsen.control/session-error
|
||||
:message "Error opening SSH session. Verify username, password, and node hostnames are correct."))))))
|
||||
|
||||
(disconnect! [this]
|
||||
(when-let [c client]
|
||||
(.close c)))
|
||||
|
||||
(execute! [this ctx action]
|
||||
; (info :permits (.availablePermits semaphore))
|
||||
(when (:dummy conn-spec)
|
||||
(throw+ {:type :jepsen.control/dummy}))
|
||||
(.acquire semaphore)
|
||||
(sshj/with-errors conn-spec ctx
|
||||
(try
|
||||
(with-open [session (.startSession client)]
|
||||
(let [cmd (.exec session (:cmd action))
|
||||
; Feed it input
|
||||
_ (when-let [input (:in action)]
|
||||
(let [stream (.getOutputStream cmd)]
|
||||
(bs/transfer input stream)
|
||||
(send-eof! client session)
|
||||
(.close stream)))
|
||||
; Read output
|
||||
out (.toString (IOUtils/readFully (.getInputStream cmd)))
|
||||
err (.toString (IOUtils/readFully (.getErrorStream cmd)))
|
||||
; Wait on command
|
||||
_ (.join cmd)]
|
||||
; Return completion
|
||||
(assoc action
|
||||
:out out
|
||||
:err err
|
||||
; There's also a .getExitErrorMessage that might be
|
||||
; interesting here?
|
||||
:exit (.getExitStatus cmd))))
|
||||
(finally
|
||||
(.release semaphore)))))
|
||||
|
||||
(upload! [this ctx local-paths remote-path _opts]
|
||||
(when (:dummy conn-spec)
|
||||
(throw+ {:type :jepsen.control/dummy}))
|
||||
(with-errors conn-spec ctx
|
||||
(with-open [sftp (.newSFTPClient client)]
|
||||
(.put sftp (FileSystemFile. local-paths) remote-path))))
|
||||
|
||||
(download! [this ctx remote-paths local-path _opts]
|
||||
(when (:dummy conn-spec)
|
||||
(throw+ {:type :jepsen.control/dummy}))
|
||||
(with-errors conn-spec ctx
|
||||
(with-open [sftp (.newSFTPClient client)]
|
||||
(.get sftp remote-paths (FileSystemFile. local-path))))))
|
||||
|
||||
(defn remote
|
||||
"Constructs an SSHJ remote."
|
||||
[]
|
||||
(-> (SSHJRemote. concurrency-limit nil nil nil)
|
||||
; We *can* use our own SCP, but shelling out is faster.
|
||||
scp/remote
|
||||
retry/remote))
|
||||
|
||||
(ns jepsen.clickhouse-keeper.main
|
||||
(:require [clojure.tools.logging :refer :all]
|
||||
[jepsen.clickhouse-keeper.utils :refer :all]
|
||||
@ -17,7 +112,6 @@
|
||||
[checker :as checker]
|
||||
[cli :as cli]
|
||||
[client :as client]
|
||||
[control :as c]
|
||||
[db :as db]
|
||||
[nemesis :as nemesis]
|
||||
[generator :as gen]
|
||||
|
@ -13,3 +13,7 @@ SET optimize_monotonous_functions_in_order_by = 1;
|
||||
EXPLAIN SYNTAX SELECT * FROM t_02147 ORDER BY toStartOfHour(date), v;
|
||||
EXPLAIN SYNTAX SELECT * FROM t_02147_dist ORDER BY toStartOfHour(date), v;
|
||||
EXPLAIN SYNTAX SELECT * FROM t_02147_merge ORDER BY toStartOfHour(date), v;
|
||||
|
||||
drop table t_02147;
|
||||
CREATE TABLE t_02147 (date DateTime, v UInt32) ENGINE = MergeTree ORDER BY date;
|
||||
select *, toString(t.v) as s from t_02147_merge as t order by date, s;
|
||||
|
@ -0,0 +1,4 @@
|
||||
CREATE TABLE test_a (id UInt32, company UInt32, total UInt64) ENGINE=SummingMergeTree() PARTITION BY company PRIMARY KEY (id) ORDER BY (id, company);
|
||||
INSERT INTO test_a SELECT number%10 as id, number%2 as company, count() as total FROM numbers(100) GROUP BY id,company;
|
||||
CREATE TABLE test_b (id UInt32, company UInt32, total UInt64) ENGINE=SummingMergeTree() PARTITION BY company ORDER BY (id, company);
|
||||
ALTER TABLE test_b REPLACE PARTITION '0' FROM test_a; -- {serverError BAD_ARGUMENTS}
|
@ -78,7 +78,7 @@ export CLICKHOUSE_PORT_POSTGRESQL=${CLICKHOUSE_PORT_POSTGRESQL:="9005"}
|
||||
export CLICKHOUSE_PORT_KEEPER=${CLICKHOUSE_PORT_KEEPER:=$(${CLICKHOUSE_EXTRACT_CONFIG} --try --key=keeper_server.tcp_port 2>/dev/null)} 2>/dev/null
|
||||
export CLICKHOUSE_PORT_KEEPER=${CLICKHOUSE_PORT_KEEPER:="9181"}
|
||||
|
||||
export CLICKHOUSE_CLIENT_SECURE=${CLICKHOUSE_CLIENT_SECURE:=$(echo "${CLICKHOUSE_CLIENT}" | sed 's/--secure //' | sed 's/'"--port=${CLICKHOUSE_PORT_TCP}"'//g; s/$/'"--secure --port=${CLICKHOUSE_PORT_TCP_SECURE}"'/g')}
|
||||
export CLICKHOUSE_CLIENT_SECURE=${CLICKHOUSE_CLIENT_SECURE:=$(echo "${CLICKHOUSE_CLIENT}" | sed 's/--secure //' | sed 's/'"--port=${CLICKHOUSE_PORT_TCP}"'//g; s/$/'"--secure --accept-invalid-certificate --port=${CLICKHOUSE_PORT_TCP_SECURE}"'/g')}
|
||||
|
||||
# Add database and log comment to url params
|
||||
if [ -v CLICKHOUSE_URL_PARAMS ]
|
||||
|
Loading…
Reference in New Issue
Block a user