Merge remote-tracking branch 'upstream/master' into fs-cache-improvement

This commit is contained in:
kssenii 2023-10-16 13:06:28 +02:00
commit 43c5e173de
548 changed files with 23892 additions and 5666 deletions

View File

@ -90,7 +90,6 @@
* Allow reading some unusual configuration of chunks from Arrow/Parquet formats. [#54370](https://github.com/ClickHouse/ClickHouse/pull/54370) ([Arthur Passos](https://github.com/arthurpassos)).
* Add `STD` alias to `stddevPop` function for MySQL compatibility. Closes [#54274](https://github.com/ClickHouse/ClickHouse/issues/54274). [#54382](https://github.com/ClickHouse/ClickHouse/pull/54382) ([Nikolay Degterinsky](https://github.com/evillique)).
* Add `addDate` function for compatibility with MySQL and `subDate` for consistency. Reference [#54275](https://github.com/ClickHouse/ClickHouse/issues/54275). [#54400](https://github.com/ClickHouse/ClickHouse/pull/54400) ([Nikolay Degterinsky](https://github.com/evillique)).
* Support `SAMPLE BY` for views. [#54477](https://github.com/ClickHouse/ClickHouse/pull/54477) ([Azat Khuzhin](https://github.com/azat)).
* Add `modification_time` into `system.detached_parts`. [#54506](https://github.com/ClickHouse/ClickHouse/pull/54506) ([Azat Khuzhin](https://github.com/azat)).
* Added a setting `splitby_max_substrings_includes_remaining_string` which controls if functions "splitBy*()" with argument "max_substring" > 0 include the remaining string (if any) in the result array (Python/Spark semantics) or not. The default behavior does not change. [#54518](https://github.com/ClickHouse/ClickHouse/pull/54518) ([Robert Schulze](https://github.com/rschu1ze)).
* Better integer types inference for `Int64`/`UInt64` fields. Continuation of [#53003](https://github.com/ClickHouse/ClickHouse/pull/53003). Now it works also for nested types like Arrays of Arrays and for functions like `map/tuple`. Issue: [#51236](https://github.com/ClickHouse/ClickHouse/issues/51236). [#54553](https://github.com/ClickHouse/ClickHouse/pull/54553) ([Kruglov Pavel](https://github.com/Avogar)).
@ -442,7 +441,7 @@
* Add new setting `disable_url_encoding` that allows to disable decoding/encoding path in uri in URL engine. [#52337](https://github.com/ClickHouse/ClickHouse/pull/52337) ([Kruglov Pavel](https://github.com/Avogar)).
#### Performance Improvement
* Enable automatic selection of the sparse serialization format by default. It improves performance. The format is supported since version 22.1. After this change, downgrading to versions older than 22.1 might not be possible. You can turn off the usage of the sparse serialization format by providing the `ratio_of_defaults_for_sparse_serialization = 1` setting for your MergeTree tables. [#49631](https://github.com/ClickHouse/ClickHouse/pull/49631) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Enable automatic selection of the sparse serialization format by default. It improves performance. The format is supported since version 22.1. After this change, downgrading to versions older than 22.1 might not be possible. A downgrade may require to set `ratio_of_defaults_for_sparse_serialization=0.9375` [55153](https://github.com/ClickHouse/ClickHouse/issues/55153). You can turn off the usage of the sparse serialization format by providing the `ratio_of_defaults_for_sparse_serialization = 1` setting for your MergeTree tables. [#49631](https://github.com/ClickHouse/ClickHouse/pull/49631) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Enable `move_all_conditions_to_prewhere` and `enable_multiple_prewhere_read_steps` settings by default. [#46365](https://github.com/ClickHouse/ClickHouse/pull/46365) ([Alexander Gololobov](https://github.com/davenger)).
* Improves performance of some queries by tuning allocator. [#46416](https://github.com/ClickHouse/ClickHouse/pull/46416) ([Azat Khuzhin](https://github.com/azat)).
* Now we use fixed-size tasks in `MergeTreePrefetchedReadPool` as in `MergeTreeReadPool`. Also from now we use connection pool for S3 requests. [#49732](https://github.com/ClickHouse/ClickHouse/pull/49732) ([Nikita Taranov](https://github.com/nickitat)).

View File

@ -26,7 +26,6 @@ curl https://clickhouse.com/ | sh
* [**ClickHouse Meetup in Amsterdam**](https://www.meetup.com/clickhouse-netherlands-user-group/events/296334590/) - Oct 31
* [**ClickHouse Meetup in Beijing**](https://www.meetup.com/clickhouse-beijing-user-group/events/296334856/) - Nov 4
* [**ClickHouse Meetup in San Francisco**](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/296334923/) - Nov 8
* [**ClickHouse Meetup in Tel Aviv**](https://www.meetup.com/clickhouse-tel-aviv-user-group/events/296487639/) - Nov 8
* [**ClickHouse Meetup in Singapore**](https://www.meetup.com/clickhouse-singapore-meetup-group/events/296334976/) - Nov 15
* [**ClickHouse Meetup in Berlin**](https://www.meetup.com/clickhouse-berlin-user-group/events/296488501/) - Nov 30
* [**ClickHouse Meetup in NYC**](https://www.meetup.com/clickhouse-new-york-user-group/events/296488779/) - Dec 11

View File

@ -70,7 +70,7 @@ namespace MongoDB
Document::Ptr queryBuildInfo(Connection & connection) const;
/// Queries server build info (all wire protocols)
Document::Ptr queryServerHello(Connection & connection) const;
Document::Ptr queryServerHello(Connection & connection, bool old = false) const;
/// Queries hello response from server (all wire protocols)
Int64 count(Connection & connection, const std::string & collectionName) const;

View File

@ -356,11 +356,19 @@ Document::Ptr Database::queryBuildInfo(Connection& connection) const
}
Document::Ptr Database::queryServerHello(Connection& connection) const
Document::Ptr Database::queryServerHello(Connection& connection, bool old) const
{
// hello can be issued on "config" system database
Poco::SharedPtr<Poco::MongoDB::QueryRequest> request = createCommand();
request->selector().add("hello", 1);
// 'hello' command was previously called 'isMaster'
std::string command_name;
if (old)
command_name = "isMaster";
else
command_name = "hello";
request->selector().add(command_name, 1);
Poco::MongoDB::ResponseMessage response;
connection.sendRequest(*request, response);

2
contrib/croaring vendored

@ -1 +1 @@
Subproject commit 3fe177c82e2b04d34b815d9295381038880cb23d
Subproject commit 9b7cc0ff1c41e9457efb6228cfd2c538d0155303

2
contrib/curl vendored

@ -1 +1 @@
Subproject commit eb3b049df526bf125eda23218e680ce7fa9ec46c
Subproject commit d755a5f7c009dd63a61b2c745180d8ba937cbfeb

View File

@ -64,6 +64,7 @@ set (SRCS
"${LIBRARY_DIR}/lib/hostsyn.c"
"${LIBRARY_DIR}/lib/hsts.c"
"${LIBRARY_DIR}/lib/http.c"
"${LIBRARY_DIR}/lib/http1.c"
"${LIBRARY_DIR}/lib/http2.c"
"${LIBRARY_DIR}/lib/http_aws_sigv4.c"
"${LIBRARY_DIR}/lib/http_chunks.c"

View File

@ -6,20 +6,14 @@ target_compile_definitions (_gtest PUBLIC GTEST_HAS_POSIX_RE=0)
target_include_directories(_gtest SYSTEM PUBLIC "${SRC_DIR}/googletest/include")
target_include_directories(_gtest PRIVATE "${SRC_DIR}/googletest")
add_library(_gtest_main "${SRC_DIR}/googletest/src/gtest_main.cc")
set_target_properties(_gtest_main PROPERTIES VERSION "1.0.0")
target_link_libraries(_gtest_main PUBLIC _gtest)
add_library(_gtest_all INTERFACE)
target_link_libraries(_gtest_all INTERFACE _gtest _gtest_main)
add_library(ch_contrib::gtest_all ALIAS _gtest_all)
add_library(ch_contrib::gtest ALIAS _gtest)
add_library(_gmock "${SRC_DIR}/googlemock/src/gmock-all.cc")
set_target_properties(_gmock PROPERTIES VERSION "1.0.0")
target_compile_definitions (_gmock PUBLIC GTEST_HAS_POSIX_RE=0)
target_include_directories(_gmock SYSTEM PUBLIC "${SRC_DIR}/googlemock/include" "${SRC_DIR}/googletest/include")
target_include_directories(_gmock PRIVATE "${SRC_DIR}/googlemock")
target_link_libraries(_gmock PUBLIC _gtest)
add_library(_gmock_main "${SRC_DIR}/googlemock/src/gmock_main.cc")
set_target_properties(_gmock_main PROPERTIES VERSION "1.0.0")

View File

@ -15,8 +15,8 @@ CLICKHOUSE_CI_LOGS_USER=${CLICKHOUSE_CI_LOGS_USER:-ci}
# Pre-configured destination cluster, where to export the data
CLICKHOUSE_CI_LOGS_CLUSTER=${CLICKHOUSE_CI_LOGS_CLUSTER:-system_logs_export}
EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime, check_name LowCardinality(String), instance_type LowCardinality(String), instance_id String, "}
EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"0 AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type, '' AS instance_id"}
EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime('UTC'), check_name String, instance_type String, instance_id String, "}
EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS UInt32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type, '' AS instance_id"}
EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "}
function __set_connection_args
@ -125,9 +125,9 @@ function setup_logs_replication
echo 'Create %_log tables'
clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table
do
# Calculate hash of its structure. Note: 1 is the version of extra columns - increment it if extra columns are changed:
# Calculate hash of its structure. Note: 4 is the version of extra columns - increment it if extra columns are changed:
hash=$(clickhouse-client --query "
SELECT sipHash64(1, groupArray((name, type)))
SELECT sipHash64(4, groupArray((name, type)))
FROM (SELECT name, type FROM system.columns
WHERE database = 'system' AND table = '$table'
ORDER BY position)

View File

@ -10,6 +10,5 @@ RUN curl -L -o /mysql-connector-j-${ver}.jar https://repo1.maven.org/maven2/com/
ENV CLASSPATH=$CLASSPATH:/mysql-connector-j-${ver}.jar
WORKDIR /jdbc
COPY Test.java Test.java
COPY PreparedStatementsTest.java PreparedStatementsTest.java
RUN javac Test.java PreparedStatementsTest.java
COPY MySQLJavaClientTest.java MySQLJavaClientTest.java
RUN javac MySQLJavaClientTest.java

View File

@ -2,7 +2,7 @@ import com.mysql.cj.MysqlType;
import java.sql.*;
public class PreparedStatementsTest {
public class MySQLJavaClientTest {
public static void main(String[] args) {
int i = 0;
String host = "127.0.0.1";
@ -10,6 +10,7 @@ public class PreparedStatementsTest {
String user = "default";
String password = "";
String database = "default";
String binary = "false";
while (i < args.length) {
switch (args[i]) {
case "--host":
@ -27,16 +28,19 @@ public class PreparedStatementsTest {
case "--database":
database = args[++i];
break;
case "--binary":
binary = args[++i];
break;
default:
i++;
break;
}
}
// useServerPrepStmts uses COM_STMT_PREPARE and COM_STMT_EXECUTE
// instead of COM_QUERY which allows us to test the binary protocol
String jdbcUrl = String.format("jdbc:mysql://%s:%s/%s?useSSL=false&useServerPrepStmts=true",
host, port, database);
// useServerPrepStmts=true -> COM_STMT_PREPARE + COM_STMT_EXECUTE -> binary
// useServerPrepStmts=false -> COM_QUERY -> text
String jdbcUrl = String.format("jdbc:mysql://%s:%s/%s?useSSL=false&useServerPrepStmts=%s",
host, port, database, binary);
try {
Class.forName("com.mysql.cj.jdbc.Driver");
@ -49,6 +53,7 @@ public class PreparedStatementsTest {
testDateTypes(conn);
testUnusualDateTime64Scales(conn);
testDateTimeTimezones(conn);
testSuspiciousNullableLowCardinalityTypes(conn);
conn.close();
} catch (Exception e) {
e.printStackTrace();
@ -58,7 +63,7 @@ public class PreparedStatementsTest {
private static void testSimpleDataTypes(Connection conn) throws SQLException {
System.out.println("### testSimpleDataTypes");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_simple_data_types").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM simple_data_types").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -83,7 +88,7 @@ public class PreparedStatementsTest {
private static void testStringTypes(Connection conn) throws SQLException {
System.out.println("### testStringTypes");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_string_types").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM string_types").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -97,7 +102,7 @@ public class PreparedStatementsTest {
private static void testLowCardinalityAndNullableTypes(Connection conn) throws SQLException {
System.out.println("### testLowCardinalityAndNullableTypes");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_low_cardinality_and_nullable_types").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM low_cardinality_and_nullable_types").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -111,7 +116,7 @@ public class PreparedStatementsTest {
private static void testDecimalTypes(Connection conn) throws SQLException {
System.out.println("### testDecimalTypes");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_decimal_types").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM decimal_types").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -127,7 +132,7 @@ public class PreparedStatementsTest {
private static void testDateTypes(Connection conn) throws SQLException {
System.out.println("### testDateTypes");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_date_types").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM date_types").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -143,7 +148,7 @@ public class PreparedStatementsTest {
private static void testUnusualDateTime64Scales(Connection conn) throws SQLException {
System.out.println("### testUnusualDateTime64Scales");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_unusual_datetime64_scales").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM unusual_datetime64_scales").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -160,7 +165,7 @@ public class PreparedStatementsTest {
private static void testDateTimeTimezones(Connection conn) throws SQLException {
System.out.println("### testDateTimeTimezones");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_datetime_timezones").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM datetime_timezones").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -172,7 +177,7 @@ public class PreparedStatementsTest {
private static void testMiscTypes(Connection conn) throws SQLException {
System.out.println("### testMiscTypes");
ResultSet rs = conn.prepareStatement("SELECT * FROM ps_misc_types").executeQuery();
ResultSet rs = conn.prepareStatement("SELECT * FROM misc_types").executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
@ -184,6 +189,20 @@ public class PreparedStatementsTest {
System.out.println();
}
private static void testSuspiciousNullableLowCardinalityTypes(Connection conn) throws SQLException {
System.out.println("### testSuspiciousNullableLowCardinalityTypes");
String query = "SELECT * FROM suspicious_nullable_low_cardinality_types";
ResultSet rs = conn.prepareStatement(query).executeQuery();
int rowNum = 1;
while (rs.next()) {
System.out.printf("Row #%d\n", rowNum++);
System.out.printf("%s, value: %s\n", getMysqlType(rs, "f"), rs.getFloat("f"));
System.out.printf("%s, value: %s\n", getMysqlType(rs, "d"), rs.getDate("d"));
System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt"), rs.getTimestamp("dt"));
}
System.out.println();
}
private static String getMysqlType(ResultSet rs, String columnLabel) throws SQLException {
ResultSetMetaData meta = rs.getMetaData();
return String.format("%s type is %s", columnLabel,

View File

@ -1,78 +0,0 @@
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
class JavaConnectorTest {
private static final String CREATE_TABLE_SQL = "CREATE TABLE IF NOT EXISTS default.test1 (`age` Int32, `name` String, `int_nullable` Nullable(Int32)) Engine = Memory";
private static final String INSERT_SQL = "INSERT INTO default.test1(`age`, `name`) VALUES(33, 'jdbc'),(44, 'ck')";
private static final String SELECT_SQL = "SELECT * FROM default.test1";
private static final String SELECT_NUMBER_SQL = "SELECT * FROM system.numbers LIMIT 13";
private static final String DROP_TABLE_SQL = "DROP TABLE default.test1";
public static void main(String[] args) {
int i = 0;
String host = "127.0.0.1";
String port = "9004";
String user = "default";
String password = "";
String database = "default";
while (i < args.length) {
switch (args[i]) {
case "--host":
host = args[++i];
break;
case "--port":
port = args[++i];
break;
case "--user":
user = args[++i];
break;
case "--password":
password = args[++i];
break;
case "--database":
database = args[++i];
break;
default:
i++;
break;
}
}
String jdbcUrl = String.format("jdbc:mysql://%s:%s/%s?useSSL=false", host, port, database);
Connection conn = null;
Statement stmt = null;
try {
Class.forName("com.mysql.cj.jdbc.Driver");
conn = DriverManager.getConnection(jdbcUrl, user, password);
stmt = conn.createStatement();
stmt.executeUpdate(CREATE_TABLE_SQL);
stmt.executeUpdate(INSERT_SQL);
ResultSet rs = stmt.executeQuery(SELECT_SQL);
while (rs.next()) {
System.out.print(rs.getString("age"));
System.out.print(rs.getString("name"));
System.out.print(rs.getString("int_nullable"));
System.out.println();
}
stmt.executeUpdate(DROP_TABLE_SQL);
rs = stmt.executeQuery(SELECT_NUMBER_SQL);
while (rs.next()) {
System.out.print(rs.getString(1));
System.out.println();
}
stmt.close();
conn.close();
} catch (Exception e) {
e.printStackTrace();
System.exit(1);
}
}
}

View File

@ -0,0 +1,16 @@
version: '2.3'
services:
openldap:
image: bitnami/openldap:2.6.6
restart: always
environment:
LDAP_ROOT: dc=example,dc=org
LDAP_ADMIN_DN: cn=admin,dc=example,dc=org
LDAP_ADMIN_USERNAME: admin
LDAP_ADMIN_PASSWORD: clickhouse
LDAP_USER_DC: users
LDAP_USERS: janedoe,johndoe
LDAP_PASSWORDS: qwerty,qwertz
LDAP_PORT_NUMBER: ${LDAP_INTERNAL_PORT:-1389}
ports:
- ${LDAP_EXTERNAL_PORT:-1389}:${LDAP_INTERNAL_PORT:-1389}

View File

@ -2,9 +2,7 @@ version: '2.3'
services:
minio1:
# Newer version of minio results in such errors:
# "AWSErrorMarshaller: Encountered AWSError 'InternalError': We encountered an internal error, please try again"
image: minio/minio:RELEASE.2021-09-23T04-46-24Z
image: minio/minio:RELEASE.2023-09-30T07-02-29Z
volumes:
- data1-1:/data1
- ${MINIO_CERTS_DIR:-}:/certs

View File

@ -2,15 +2,13 @@ version: '2.3'
services:
rabbitmq1:
image: rabbitmq:3.8-management-alpine
image: rabbitmq:3.12.6-management-alpine
hostname: rabbitmq1
expose:
- ${RABBITMQ_PORT:-5672}
environment:
RABBITMQ_DEFAULT_USER: "root"
RABBITMQ_DEFAULT_PASS: "clickhouse"
RABBITMQ_LOG_BASE: /rabbitmq_logs/
volumes:
- type: ${RABBITMQ_LOGS_FS:-tmpfs}
source: ${RABBITMQ_LOGS:-}
target: /rabbitmq_logs/
- "${RABBITMQ_COOKIE_FILE}:/var/lib/rabbitmq/.erlang.cookie"
- /misc/rabbitmq.conf:/etc/rabbitmq/rabbitmq.conf

View File

@ -0,0 +1,8 @@
loopback_users.guest = false
listeners.tcp.default = 5672
default_pass = clickhouse
default_user = root
management.tcp.port = 15672
log.file = /rabbitmq_logs/rabbit.log
log.file.level = debug

View File

@ -60,7 +60,7 @@ Before using cache, add it to `config.xml`
- limit_size: Required. The maximum size(in bytes) of local cache files.
- bytes_read_before_flush: Control bytes before flush to local filesystem when downloading file from remote filesystem. The default value is 1MB.
When ClickHouse is started up with local cache for remote filesystem enabled, users can still choose not to use cache with `settings use_local_cache_for_remote_fs = 0` in their query. `use_local_cache_for_remote_fs` is `false` in default.
When ClickHouse is started up with local cache for remote filesystem enabled, users can still choose not to use cache with `settings use_local_cache_for_remote_storage = 0` in their query. `use_local_cache_for_remote_storage` is `1` by default.
### Query Hive Table with ORC Input Format

View File

@ -37,8 +37,8 @@ The [Merge](/docs/en/engines/table-engines/special/merge.md/#merge) engine does
``` sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
(
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr1] [TTL expr1] [CODEC(codec1)] [[NOT] NULL|PRIMARY KEY],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr2] [TTL expr2] [CODEC(codec2)] [[NOT] NULL|PRIMARY KEY],
name1 [type1] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr1] [COMMENT ...] [CODEC(codec1)] [TTL expr1] [PRIMARY KEY],
name2 [type2] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr2] [COMMENT ...] [CODEC(codec2)] [TTL expr2] [PRIMARY KEY],
...
INDEX index_name1 expr1 TYPE type1(...) [GRANULARITY value1],
INDEX index_name2 expr2 TYPE type2(...) [GRANULARITY value2],

View File

@ -100,8 +100,41 @@ SELECT * FROM mySecondReplacingMT FINAL;
The row is deleted when `OPTIMIZE ... FINAL CLEANUP` or `OPTIMIZE ... FINAL` is used, or if the engine setting `clean_deleted_rows` has been set to `Always`.
No matter the operation on the data, the version must be increased. If two inserted rows have the same version number, the last inserted row is the one kept.
:::
Example:
```sql
-- with ver and is_deleted
CREATE OR REPLACE TABLE myThirdReplacingMT
(
`key` Int64,
`someCol` String,
`eventTime` DateTime,
`is_deleted` UInt8
)
ENGINE = ReplacingMergeTree(eventTime, is_deleted)
ORDER BY key;
INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 01:01:01', 0);
INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 01:01:01', 1);
select * from myThirdReplacingMT final;
0 rows in set. Elapsed: 0.003 sec.
-- delete rows with is_deleted
OPTIMIZE TABLE myThirdReplacingMT FINAL CLEANUP;
INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 00:00:00', 0);
select * from myThirdReplacingMT final;
┌─key─┬─someCol─┬───────────eventTime─┬─is_deleted─┐
│ 1 │ first │ 2020-01-01 00:00:00 │ 0 │
└─────┴─────────┴─────────────────────┴────────────┘
```
## Query clauses
When creating a `ReplacingMergeTree` table the same [clauses](../../../engines/table-engines/mergetree-family/mergetree.md) are required, as when creating a `MergeTree` table.

View File

@ -1924,6 +1924,14 @@ SELECT * FROM test.hits format Protobuf SETTINGS format_protobuf_use_autogenerat
In this case autogenerated Protobuf schema will be saved in file `path/to/schema/schema.capnp`.
### Drop Protobuf cache
To reload Protobuf schema loaded from [format_schema_path](../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-format_schema_path) use [SYSTEM DROP ... FORMAT CACHE](../sql-reference/statements/system.md/#system-drop-schema-format) statement.
```sql
SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf
```
## ProtobufSingle {#protobufsingle}
Same as [Protobuf](#protobuf) but for storing/parsing single Protobuf message without length delimiters.

View File

@ -58,7 +58,7 @@ Connection: Close
Content-Type: text/tab-separated-values; charset=UTF-8
X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal
X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f
X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334,"peak_memory_usage":"0"}
X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
1
```
@ -288,9 +288,9 @@ Similarly, you can use ClickHouse sessions in the HTTP protocol. To do this, you
You can receive information about the progress of a query in `X-ClickHouse-Progress` response headers. To do this, enable [send_progress_in_http_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Example of the header sequence:
``` text
X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","elapsed_ns":"662334","peak_memory_usage":"4371480"}
X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","elapsed_ns":"992334","peak_memory_usage":"13621616"}
X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","elapsed_ns":"1232334","peak_memory_usage":"23155600"}
X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","elapsed_ns":"662334"}
X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","elapsed_ns":"992334"}
X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","elapsed_ns":"1232334"}
```
Possible header fields:
@ -439,7 +439,7 @@ $ curl -v 'http://localhost:8123/predefined_query'
< X-ClickHouse-Format: Template
< X-ClickHouse-Timezone: Asia/Shanghai
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
# HELP "Query" "Number of executing queries"
# TYPE "Query" counter
@ -604,7 +604,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi'
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
* Connection #0 to host localhost left intact
Say Hi!%
@ -644,7 +644,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler'
< Content-Type: text/plain; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
* Connection #0 to host localhost left intact
<html ng-app="SMI2"><head><base href="http://ui.tabix.io/"></head><body><div ui-view="" class="content-ui"></div><script src="http://loader.tabix.io/master.js"></script></body></html>%
@ -696,7 +696,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler'
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
<html><body>Absolute Path File</body></html>
* Connection #0 to host localhost left intact
@ -715,7 +715,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler'
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
<html><body>Relative Path File</body></html>
* Connection #0 to host localhost left intact
@ -800,4 +800,3 @@ $ curl 'http://localhost:8123/?query=SELECT+number,+throwIf(number>2)+from+syste
<exception>Code: 395. DB::Exception: Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 2) :: 2) -> throwIf(greater(number, 2)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) (version 23.8.1.1)</exception>
</result>
```

View File

@ -10,10 +10,6 @@ ClickHouse supports the MySQL wire protocol. This allow tools that are MySQL-com
## Enabling the MySQL Interface On ClickHouse Cloud
:::note
The MySQL interface for ClickHouse Cloud is currently in private preview. Please contact support@clickhouse.com to enable this feature for your ClickHouse Cloud service.
:::
1. After creating your ClickHouse Cloud Service, on the credentials screen, select the MySQL tab
![Credentials screen - Prompt](./images/mysql1.png)

View File

@ -44,6 +44,8 @@ ClickHouse Inc does **not** maintain the libraries listed below and hasnt don
- [nestjs-clickhouse](https://github.com/depyronick/nestjs-clickhouse)
- [clickhouse-client](https://github.com/depyronick/clickhouse-client)
- [node-clickhouse-orm](https://github.com/zimv/node-clickhouse-orm)
- [clickhouse-ts](https://github.com/bytadaniel/clickhouse-ts)
- [clickcache](https://github.com/bytadaniel/clickcache)
### Perl
- [perl-DBD-ClickHouse](https://github.com/elcamlost/perl-DBD-ClickHouse)
- [HTTP-ClickHouse](https://metacpan.org/release/HTTP-ClickHouse)

View File

@ -406,7 +406,7 @@ RESTORE TABLE data AS data_restored FROM Disk('s3_plain', 'cloud_backup');
:::note
But keep in mind that:
- This disk should not be used for `MergeTree` itself, only for `BACKUP`/`RESTORE`
- It has excessive API calls
- If your tables are backed by S3 storage, it doesn't use `CopyObject` calls to copy parts to the destination bucket, instead, it downloads and uploads them, which is very inefficient. Prefer to use `BACKUP ... TO S3(<endpoint>)` syntax for this use-case.
:::
## Alternatives

View File

@ -61,17 +61,17 @@ FROM table
SETTINGS use_query_cache = true, enable_writes_to_query_cache = false;
```
For maximum control, it is generally recommended to provide settings "use_query_cache", "enable_writes_to_query_cache" and
"enable_reads_from_query_cache" only with specific queries. It is also possible to enable caching at user or profile level (e.g. via `SET
For maximum control, it is generally recommended to provide settings `use_query_cache`, `enable_writes_to_query_cache` and
`enable_reads_from_query_cache` only with specific queries. It is also possible to enable caching at user or profile level (e.g. via `SET
use_query_cache = true`) but one should keep in mind that all `SELECT` queries including monitoring or debugging queries to system tables
may return cached results then.
The query cache can be cleared using statement `SYSTEM DROP QUERY CACHE`. The content of the query cache is displayed in system table
`system.query_cache`. The number of query cache hits and misses since database start are shown as events "QueryCacheHits" and
"QueryCacheMisses" in system table [system.events](system-tables/events.md). Both counters are only updated for `SELECT` queries which run
with setting `use_query_cache = true`, other queries do not affect "QueryCacheMisses". Field `query_cache_usage` in system table
[system.query_log](system-tables/query_log.md) shows for each executed query whether the query result was written into or read from the
query cache. Asynchronous metrics "QueryCacheEntries" and "QueryCacheBytes" in system table
[system.query_cache](system-tables/query_cache.md). The number of query cache hits and misses since database start are shown as events
"QueryCacheHits" and "QueryCacheMisses" in system table [system.events](system-tables/events.md). Both counters are only updated for
`SELECT` queries which run with setting `use_query_cache = true`, other queries do not affect "QueryCacheMisses". Field `query_cache_usage`
in system table [system.query_log](system-tables/query_log.md) shows for each executed query whether the query result was written into or
read from the query cache. Asynchronous metrics "QueryCacheEntries" and "QueryCacheBytes" in system table
[system.asynchronous_metrics](system-tables/asynchronous_metrics.md) show how many entries / bytes the query cache currently contains.
The query cache exists once per ClickHouse server process. However, cache results are by default not shared between users. This can be
@ -86,9 +86,18 @@ If the query was aborted due to an exception or user cancellation, no entry is w
The size of the query cache in bytes, the maximum number of cache entries and the maximum size of individual cache entries (in bytes and in
records) can be configured using different [server configuration options](server-configuration-parameters/settings.md#server_configuration_parameters_query-cache).
```xml
<query_cache>
<max_size_in_bytes>1073741824</max_size_in_bytes>
<max_entries>1024</max_entries>
<max_entry_size_in_bytes>1048576</max_entry_size_in_bytes>
<max_entry_size_in_rows>30000000</max_entry_size_in_rows>
</query_cache>
```
It is also possible to limit the cache usage of individual users using [settings profiles](settings/settings-profiles.md) and [settings
constraints](settings/constraints-on-settings.md). More specifically, you can restrict the maximum amount of memory (in bytes) a user may
allocate in the query cache and the the maximum number of stored query results. For that, first provide configurations
allocate in the query cache and the maximum number of stored query results. For that, first provide configurations
[query_cache_max_size_in_bytes](settings/settings.md#query-cache-max-size-in-bytes) and
[query_cache_max_entries](settings/settings.md#query-cache-size-max-entries) in a user profile in `users.xml`, then make both settings
readonly:
@ -158,6 +167,7 @@ Also, results of queries with non-deterministic functions are not cached by defa
- functions which depend on the environment: [`currentUser()`](../sql-reference/functions/other-functions.md#currentUser),
[`queryID()`](../sql-reference/functions/other-functions.md#queryID),
[`getMacro()`](../sql-reference/functions/other-functions.md#getMacro) etc.
To force caching of results of queries with non-deterministic functions regardless, use setting
[query_cache_store_results_of_queries_with_nondeterministic_functions](settings/settings.md#query-cache-store-results-of-queries-with-nondeterministic-functions).

View File

@ -2403,7 +2403,8 @@ This section contains the following parameters:
- zookeeper_load_balancing - Specifies the algorithm of ZooKeeper node selection.
* random - randomly selects one of ZooKeeper nodes.
* in_order - selects the first ZooKeeper node, if it's not available then the second, and so on.
* nearest_hostname - selects a ZooKeeper node with a hostname that is most similar to the servers hostname.
* nearest_hostname - selects a ZooKeeper node with a hostname that is most similar to the servers hostname, hostname is compared with name prefix.
* hostname_levenshtein_distance - just like nearest_hostname, but it compares hostname in a levenshtein distance manner.
* first_or_random - selects the first ZooKeeper node, if it's not available then randomly selects one of remaining ZooKeeper nodes.
* round_robin - selects the first ZooKeeper node, if reconnection happens selects the next.
@ -2425,7 +2426,7 @@ This section contains the following parameters:
<root>/path/to/zookeeper/node</root>
<!-- Optional. Zookeeper digest ACL string. -->
<identity>user:password</identity>
<!--<zookeeper_load_balancing>random / in_order / nearest_hostname / first_or_random / round_robin</zookeeper_load_balancing>-->
<!--<zookeeper_load_balancing>random / in_order / nearest_hostname / hostname_levenshtein_distance / first_or_random / round_robin</zookeeper_load_balancing>-->
<zookeeper_load_balancing>random</zookeeper_load_balancing>
</zookeeper>
```

View File

@ -502,7 +502,7 @@ Possible values:
Default value: 480.
After merging several parts into a new part, ClickHouse marks the original parts as inactive and deletes them only after `old_parts_lifetime` seconds.
Inactive parts are removed if they are not used by current queries, i.e. if the `refcount` of the part is zero.
Inactive parts are removed if they are not used by current queries, i.e. if the `refcount` of the part is 1.
`fsync` is not called for new parts, so for some time new parts exist only in the server's RAM (OS cache). If the server is rebooted spontaneously, new parts can be lost or damaged.
To protect data inactive parts are not deleted immediately.

View File

@ -1413,6 +1413,7 @@ ClickHouse supports the following algorithms of choosing replicas:
- [Random](#load_balancing-random) (by default)
- [Nearest hostname](#load_balancing-nearest_hostname)
- [Hostname levenshtein distance](#load_balancing-hostname_levenshtein_distance)
- [In order](#load_balancing-in_order)
- [First or random](#load_balancing-first_or_random)
- [Round robin](#load_balancing-round_robin)
@ -1444,6 +1445,25 @@ This method might seem primitive, but it does not require external data about ne
Thus, if there are equivalent replicas, the closest one by name is preferred.
We can also assume that when sending a query to the same server, in the absence of failures, a distributed query will also go to the same servers. So even if different data is placed on the replicas, the query will return mostly the same results.
### Hostname levenshtein distance {#load_balancing-hostname_levenshtein_distance}
``` sql
load_balancing = hostname_levenshtein_distance
```
Just like `nearest_hostname`, but it compares hostname in a [levenshtein distance](https://en.wikipedia.org/wiki/Levenshtein_distance) manner. For example:
``` text
example-clickhouse-0-0 ample-clickhouse-0-0
1
example-clickhouse-0-0 example-clickhouse-1-10
2
example-clickhouse-0-0 example-clickhouse-12-0
3
```
### In Order {#load_balancing-in_order}
``` sql
@ -3279,6 +3299,17 @@ Possible values:
Default value: `0`.
## use_mysql_types_in_show_columns {#use_mysql_types_in_show_columns}
Show the names of MySQL data types corresponding to ClickHouse data types in [SHOW COLUMNS](../../sql-reference/statements/show.md#show_columns).
Possible values:
- 0 - Show names of native ClickHouse data types.
- 1 - Show names of MySQL data types corresponding to ClickHouse data types.
Default value: `0`.
## execute_merges_on_single_replica_time_threshold {#execute-merges-on-single-replica-time-threshold}
Enables special logic to perform merges on replicas.

View File

@ -69,6 +69,7 @@ Columns:
- `domain_catalog` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported.
- `domain_schema` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported.
- `domain_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported.
- `extra` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `STORED GENERATED` for `MATERIALIZED`-type columns, `VIRTUAL GENERATED` for `ALIAS`-type columns, `DEFAULT_GENERATED` for `DEFAULT`-type columns, or `NULL`.
**Example**
@ -198,6 +199,8 @@ Columns:
- `FOREIGN TABLE`
- `LOCAL TEMPORARY`
- `SYSTEM VIEW`
- `table_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The total
number of rows. NULL if it could not be determined.
- `data_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The size of
the data on-disk. NULL if it could not be determined.
- `table_collation` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — The table default collation. Always `utf8mb4_0900_ai_ci`.

View File

@ -11,21 +11,21 @@ The `system.part_log` table contains the following columns:
- `query_id` ([String](../../sql-reference/data-types/string.md)) — Identifier of the `INSERT` query that created this data part.
- `event_type` ([Enum8](../../sql-reference/data-types/enum.md)) — Type of the event that occurred with the data part. Can have one of the following values:
- `NEW_PART` — Inserting of a new data part.
- `MERGE_PARTS` — Merging of data parts.
- `DOWNLOAD_PART` — Downloading a data part.
- `REMOVE_PART` — Removing or detaching a data part using [DETACH PARTITION](../../sql-reference/statements/alter/partition.md#alter_detach-partition).
- `MUTATE_PART` — Mutating of a data part.
- `MOVE_PART` — Moving the data part from the one disk to another one.
- `NewPart` — Inserting of a new data part.
- `MergeParts` — Merging of data parts.
- `DownloadParts` — Downloading a data part.
- `RemovePart` — Removing or detaching a data part using [DETACH PARTITION](../../sql-reference/statements/alter/partition.md#alter_detach-partition).
- `MutatePart` — Mutating of a data part.
- `MovePart` — Moving the data part from the one disk to another one.
- `merge_reason` ([Enum8](../../sql-reference/data-types/enum.md)) — The reason for the event with type `MERGE_PARTS`. Can have one of the following values:
- `NOT_A_MERGE` — The current event has the type other than `MERGE_PARTS`.
- `REGULAR_MERGE` — Some regular merge.
- `TTL_DELETE_MERGE` — Cleaning up expired data.
- `TTL_RECOMPRESS_MERGE` — Recompressing data part with the.
- `NotAMerge` — The current event has the type other than `MERGE_PARTS`.
- `RegularMerge` — Some regular merge.
- `TTLDeleteMerge` — Cleaning up expired data.
- `TTLRecompressMerge` — Recompressing data part with the.
- `merge_algorithm` ([Enum8](../../sql-reference/data-types/enum.md)) — Merge algorithm for the event with type `MERGE_PARTS`. Can have one of the following values:
- `UNDECIDED`
- `HORIZONTAL`
- `VERTICAL`
- `Undecided`
- `Horizontal`
- `Vertical`
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time.
- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Event time with microseconds precision.

View File

@ -0,0 +1,36 @@
---
slug: /en/operations/system-tables/query_cache
---
# query_cache
Shows the content of the [query cache](../query-cache.md).
Columns:
- `query` ([String](../../sql-reference/data-types/string.md)) — Query string.
- `result_size` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Size of the query cache entry.
- `stale` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is stale.
- `shared` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is shared between multiple users.
- `compressed` ([UInt8](../../sql-reference/data-types/int-uint.md)) — If the query cache entry is compressed.
- `expires_at` ([DateTime](../../sql-reference/data-types/datetime.md)) — When the query cache entry becomes stale.
- `key_hash` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — A hash of the query string, used as a key to find query cache entries.
**Example**
``` sql
SELECT * FROM system.query_cache FORMAT Vertical;
```
``` text
Row 1:
──────
query: SELECT 1 SETTINGS use_query_cache = 1
result_size: 128
stale: 0
shared: 0
compressed: 1
expires_at: 2023-10-13 13:35:45
key_hash: 12188185624808016954
1 row in set. Elapsed: 0.004 sec.
```

View File

@ -55,6 +55,7 @@ keeper foo bar
- `rmr <path>` -- Recursively deletes path. Confirmation required
- `flwc <command>` -- Executes four-letter-word command
- `help` -- Prints this message
- `get_all_children_number [path]` -- Get all numbers of children nodes under a specific path
- `get_stat [path]` -- Returns the node's stat (default `.`)
- `find_super_nodes <threshold> [path]` -- Finds nodes with number of children larger than some threshold for the given path (default `.`)
- `delete_stale_backups` -- Deletes ClickHouse nodes used for backups that are now inactive

View File

@ -104,7 +104,7 @@ SELECT argMin((a, b), (b, a)), min(tuple(b, a)) FROM test;
└──────────────────────────────────┴──────────────────┘
SELECT argMin(a, tuple(b)) FROM test;
┌─argMax(a, tuple(b))─┐
┌─argMin(a, tuple(b))─┐
│ d │ -- `Tuple` can be used in `min` to not skip rows with `NULL` values as b.
└─────────────────────┘
```

View File

@ -12,7 +12,7 @@ Tuples are used for temporary column grouping. Columns can be grouped when an IN
Tuples can be the result of a query. In this case, for text formats other than JSON, values are comma-separated in brackets. In JSON formats, tuples are output as arrays (in square brackets).
## Creating a Tuple
## Creating Tuples
You can use a function to create a tuple:
@ -23,7 +23,7 @@ tuple(T1, T2, ...)
Example of creating a tuple:
``` sql
SELECT tuple(1,'a') AS x, toTypeName(x)
SELECT tuple(1, 'a') AS x, toTypeName(x)
```
``` text
@ -32,7 +32,7 @@ SELECT tuple(1,'a') AS x, toTypeName(x)
└─────────┴───────────────────────────┘
```
Tuple can contain a single element
A Tuple can contain a single element
Example:
@ -46,12 +46,12 @@ SELECT tuple('a') AS x;
└───────┘
```
There is a syntax sugar using parentheses `( tuple_element1, tuple_element2 )` to create a tuple of several elements without tuple function.
Syntax `(tuple_element1, tuple_element2)` may be used to create a tuple of several elements without calling the `tuple()` function.
Example:
``` sql
SELECT (1, 'a') AS x, (today(), rand(), 'someString') y, ('a') not_a_tuple;
SELECT (1, 'a') AS x, (today(), rand(), 'someString') AS y, ('a') AS not_a_tuple;
```
``` text
@ -60,9 +60,9 @@ SELECT (1, 'a') AS x, (today(), rand(), 'someString') y, ('a') not_a_tuple;
└─────────┴────────────────────────────────────────┴─────────────┘
```
## Working with Data Types
## Data Type Detection
When creating a tuple on the fly, ClickHouse automatically detects the type of each argument as the minimum of the types which can store the argument value. If the argument is [NULL](../../sql-reference/syntax.md#null-literal), the type of the tuple element is [Nullable](../../sql-reference/data-types/nullable.md).
When creating tuples on the fly, ClickHouse interferes the type of the tuples arguments as the smallest types which can hold the provided argument value. If the value is [NULL](../../sql-reference/syntax.md#null-literal), the interfered type is [Nullable](../../sql-reference/data-types/nullable.md).
Example of automatic data type detection:
@ -71,23 +71,21 @@ SELECT tuple(1, NULL) AS x, toTypeName(x)
```
``` text
┌─x────────┬─toTypeName(tuple(1, NULL))──────┐
│ (1,NULL) │ Tuple(UInt8, Nullable(Nothing)) │
└──────────┴─────────────────────────────────┘
┌─x────────┬─toTypeName(tuple(1, NULL))──────┐
│ (1, NULL) │ Tuple(UInt8, Nullable(Nothing)) │
└──────────┴─────────────────────────────────┘
```
## Addressing Tuple Elements
## Referring to Tuple Elements
It is possible to read elements of named tuples using indexes and names:
Tuple elements can be referred to by name or by index:
``` sql
CREATE TABLE named_tuples (`a` Tuple(s String, i Int64)) ENGINE = Memory;
INSERT INTO named_tuples VALUES (('y', 10)), (('x',-10));
SELECT a.s FROM named_tuples;
SELECT a.2 FROM named_tuples;
SELECT a.s FROM named_tuples; -- by name
SELECT a.2 FROM named_tuples; -- by index
```
Result:
@ -106,7 +104,7 @@ Result:
## Comparison operations with Tuple
The operation of comparing two tuples is performed sequentially element by element from left to right. If the element of the first tuple is greater than the corresponding element of the second tuple, then the first tuple is greater than the second, if the elements are equal, the next element is compared.
Two tuples are compared by sequentially comparing their elements from the left to the right. If first tuples element is greater (smaller) than the second tuples corresponding element, then the first tuple is greater (smaller) than the second, otherwise (both elements are equal), the next element is compared.
Example:

View File

@ -73,7 +73,7 @@ Calculates the product of two values `a` and `b`.
multiply(a, b)
```
Alias: `a \* b` (operator)
Alias: `a * b` (operator)
## divide
@ -345,9 +345,9 @@ Result:
┌─multiply(toDecimal64(-12.647, 3), toDecimal32(2.1239, 4))─┐
│ -26.8609633 │
└───────────────────────────────────────────────────────────┘
┌─multiplyDecimal(toDecimal64(-12.647, 3), toDecimal32(2.1239, 4))─┐
│ -26.8609 │
└──────────────────────────────────────────────────────────────────┘
┌───────a─┬──────b─┬─multiplyDecimal(toDecimal64(-12.647, 3), toDecimal32(2.1239, 4))─┐
│ -12.647 │ 2.1239 │ -26.8609 │
└─────────┴────────┴──────────────────────────────────────────────────────────────────┘
```
```sql
@ -441,3 +441,40 @@ DB::Exception: Decimal result's scale is less than argument's one: While process
│ -12 │ 2.1 │ -5.7 │ -5.71428 │
└─────┴─────┴────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────┘
```
## byteSwap
Reverses the bytes of an integer, i.e. changes its [endianness](https://en.wikipedia.org/wiki/Endianness).
**Syntax**
```sql
byteSwap(a)
```
**Example**
```sql
byteSwap(3351772109)
```
Result:
```result
┌─byteSwap(3351772109)─┐
│ 3455829959 │
└──────────────────────┘
```
The above example can be worked out in the following manner:
1. Convert the base-10 integer to its equivalent hexadecimal format in big-endian format, i.e. 3351772109 -> C7 C7 FB CD (4 bytes)
2. Reverse the bytes, i.e. C7 C7 FB CD -> CD FB C7 C7
3. Convert the result back to an integer assuming big-endian, i.e. CD FB C7 C7 -> 3455829959
One use case of this function is reversing IPv4s:
```result
┌─toIPv4(byteSwap(toUInt32(toIPv4('205.251.199.199'))))─┐
│ 199.199.251.205 │
└───────────────────────────────────────────────────────┘
```

View File

@ -1123,6 +1123,32 @@ Result:
└─────────────────────────────┘
```
## arrayFold
Applies a lambda function to one or more equally-sized arrays and collects the result in an accumulator.
**Syntax**
``` sql
arrayFold(lambda_function, arr1, arr2, ..., accumulator)
```
**Example**
Query:
``` sql
SELECT arrayFold( x,acc -> acc + x*2, [1, 2, 3, 4], toInt64(3)) AS res;
```
Result:
``` text
┌─arrayFold(lambda(tuple(x, acc), plus(acc, multiply(x, 2))), [1, 2, 3, 4], toInt64(3))─┐
│ 3 │
└───────────────────────────────────────────────────────────────────────────────────────┘
```
## arrayReverse(arr)
Returns an array of the same size as the original array containing the elements in reverse order.
@ -2118,6 +2144,80 @@ Result:
└─────────────────────┘
```
## arrayRandomSample
Function `arrayRandomSample` returns a subset with `samples`-many random elements of an input array. If `samples` exceeds the size of the input array, the sample size is limited to the size of the array. In this case, all elements of the input array are returned, but the order is not guaranteed. The function can handle both flat arrays and nested arrays.
**Syntax**
```sql
arrayRandomSample(arr, samples)
```
**Arguments**
- `arr` — The input array from which to sample elements. This may be flat or nested arrays.
- `samples` — An unsigned integer specifying the number of elements to include in the random sample.
**Returned Value**
- An array containing a random sample of elements from the input array.
**Examples**
Query:
```sql
SELECT arrayRandomSample(['apple', 'banana', 'cherry', 'date'], 2) as res;
```
Result:
```
┌─res────────────────┐
│ ['banana','apple'] │
└────────────────────┘
```
Query:
```sql
SELECT arrayRandomSample([[1, 2], [3, 4], [5, 6]], 2) as res;
```
Result:
```
┌─res───────────┐
│ [[3,4],[5,6]] │
└───────────────┘
```
Query:
```sql
SELECT arrayRandomSample([1, 2, 3, 4, 5], 0) as res;
```
Result:
```
┌─res─┐
│ [] │
└─────┘
```
Query:
```sql
SELECT arrayRandomSample([1, 2, 3], 5) as res;
```
Result:
```
┌─res─────┐
│ [3,1,2] │
└─────────┘
```
## Distance functions
All supported functions are described in [distance functions documentation](../../sql-reference/functions/distance-functions.md).

File diff suppressed because it is too large Load Diff

View File

@ -68,6 +68,45 @@ WHERE macro = 'test';
└───────┴──────────────┘
```
## getHttpHeader
Returns the value of specified http header.If there is no such header or the request method is not http, it will return empty string.
**Syntax**
```sql
getHttpHeader(name);
```
**Arguments**
- `name` — Http header name .[String](../../sql-reference/data-types/string.md#string)
**Returned value**
Value of the specified header.
Type:[String](../../sql-reference/data-types/string.md#string).
When we use `clickhouse-client` to execute this function, we'll always get empty string, because client doesn't use http protocol.
```sql
SELECT getHttpHeader('test')
```
result:
```text
┌─getHttpHeader('test')─┐
│ │
└───────────────────────┘
```
Try to use http request:
```shell
echo "select getHttpHeader('X-Clickhouse-User')" | curl -H 'X-ClickHouse-User: default' -H 'X-ClickHouse-Key: ' 'http://localhost:8123/' -d @-
#result
default
```
## FQDN
Returns the fully qualified domain name of the ClickHouse server.

View File

@ -34,10 +34,10 @@ Returns an array of selected substrings. Empty substrings may be selected when:
Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)).
:::note
The behavior of parameter `max_substrings` changed starting with ClickHouse v22.11. In versions older than that, `max_substrings` > 0 meant that `max_substring`-many splits were performed and that the remainder of the string was returned as the final element of the list.
The behavior of parameter `max_substrings` changed starting with ClickHouse v22.11. In versions older than that, `max_substrings > 0` meant that `max_substring`-many splits were performed and that the remainder of the string was returned as the final element of the list.
For example,
- in v22.10: `SELECT splitByChar('=', 'a=b=c=d', 2); -- ['a','b','c=d']`
- in v22.11: `SELECT splitByChar('=', 'a=b=c=d', 2); -- ['a','b']`
- in v22.10: `SELECT splitByChar('=', 'a=b=c=d', 2);` returned `['a','b','c=d']`
- in v22.11: `SELECT splitByChar('=', 'a=b=c=d', 2);` returned `['a','b']`
A behavior similar to ClickHouse pre-v22.11 can be achieved by setting
[splitby_max_substrings_includes_remaining_string](../../operations/settings/settings.md#splitby_max_substrings_includes_remaining_string)

View File

@ -37,6 +37,8 @@ tupleElement(tuple, name, [, default_value])
Performs syntactic substitution of [tuple](../../sql-reference/data-types/tuple.md#tuplet1-t2) elements in the call location.
The names of the result columns are implementation-specific and subject to change. Do not assume specific column names after `untuple`.
**Syntax**
``` sql
@ -87,8 +89,6 @@ Result:
└───────┴───────┘
```
Note: the names are implementation specific and are subject to change. You should not assume specific names of the columns after application of the `untuple`.
Example of using an `EXCEPT` expression:
Query:

View File

@ -487,7 +487,7 @@ ClickHouse supports temporary tables which have the following characteristics:
- The DB cant be specified for a temporary table. It is created outside of databases.
- Impossible to create a temporary table with distributed DDL query on all cluster servers (by using `ON CLUSTER`): this table exists only in the current session.
- If a temporary table has the same name as another one and a query specifies the table name without specifying the DB, the temporary table will be used.
- For distributed query processing, temporary tables used in a query are passed to remote servers.
- For distributed query processing, temporary tables with Memory engine used in a query are passed to remote servers.
To create a temporary table, use the following syntax:

View File

@ -189,7 +189,7 @@ Result:
- [Create Tables](https://clickhouse.com/docs/en/getting-started/tutorial/#create-tables)
- [SHOW CREATE TABLE](https://clickhouse.com/docs/en/sql-reference/statements/show/#show-create-table)
## SHOW COLUMNS
## SHOW COLUMNS {#show_columns}
Displays a list of columns
@ -206,15 +206,15 @@ The optional keyword `EXTENDED` currently has no effect, it only exists for MySQ
The optional keyword `FULL` causes the output to include the collation, comment and privilege columns.
The statement produces a result table with the following structure:
- field - The name of the column (String)
- type - The column data type (String)
- null - `YES` if the column data type is Nullable, `NO` otherwise (String)
- key - `PRI` if the column is part of the primary key, `SOR` if the column is part of the sorting key, empty otherwise (String)
- default - Default expression of the column if it is of type `ALIAS`, `DEFAULT`, or `MATERIALIZED`, otherwise `NULL`. (Nullable(String))
- extra - Additional information, currently unused (String)
- collation - (only if `FULL` keyword was specified) Collation of the column, always `NULL` because ClickHouse has no per-column collations (Nullable(String))
- comment - (only if `FULL` keyword was specified) Comment on the column (String)
- privilege - (only if `FULL` keyword was specified) The privilege you have on this column, currently not available (String)
- `field` - The name of the column (String)
- `type` - The column data type. If setting `[use_mysql_types_in_show_columns](../../operations/settings/settings.md#use_mysql_types_in_show_columns) = 1` (default: 0), then the equivalent type name in MySQL is shown. (String)
- `null` - `YES` if the column data type is Nullable, `NO` otherwise (String)
- `key` - `PRI` if the column is part of the primary key, `SOR` if the column is part of the sorting key, empty otherwise (String)
- `default` - Default expression of the column if it is of type `ALIAS`, `DEFAULT`, or `MATERIALIZED`, otherwise `NULL`. (Nullable(String))
- `extra` - Additional information, currently unused (String)
- `collation` - (only if `FULL` keyword was specified) Collation of the column, always `NULL` because ClickHouse has no per-column collations (Nullable(String))
- `comment` - (only if `FULL` keyword was specified) Comment on the column (String)
- `privilege` - (only if `FULL` keyword was specified) The privilege you have on this column, currently not available (String)
**Examples**
@ -286,21 +286,21 @@ equivalent. If no database is specified, the query assumes the current database
The optional keyword `EXTENDED` currently has no effect, it only exists for MySQL compatibility.
The statement produces a result table with the following structure:
- table - The name of the table. (String)
- non_unique - Always `1` as ClickHouse does not support uniqueness constraints. (UInt8)
- key_name - The name of the index, `PRIMARY` if the index is a primary key index. (String)
- seq_in_index - For a primary key index, the position of the column starting from `1`. For a data skipping index: always `1`. (UInt8)
- column_name - For a primary key index, the name of the column. For a data skipping index: `''` (empty string), see field "expression". (String)
- collation - The sorting of the column in the index: `A` if ascending, `D` if descending, `NULL` if unsorted. (Nullable(String))
- cardinality - An estimation of the index cardinality (number of unique values in the index). Currently always 0. (UInt64)
- sub_part - Always `NULL` because ClickHouse does not support index prefixes like MySQL. (Nullable(String))
- packed - Always `NULL` because ClickHouse does not support packed indexes (like MySQL). (Nullable(String))
- null - Currently unused
- index_type - The index type, e.g. `PRIMARY`, `MINMAX`, `BLOOM_FILTER` etc. (String)
- comment - Additional information about the index, currently always `''` (empty string). (String)
- index_comment - `''` (empty string) because indexes in ClickHouse cannot have a `COMMENT` field (like in MySQL). (String)
- visible - If the index is visible to the optimizer, always `YES`. (String)
- expression - For a data skipping index, the index expression. For a primary key index: `''` (empty string). (String)
- `table` - The name of the table. (String)
- `non_unique` - Always `1` as ClickHouse does not support uniqueness constraints. (UInt8)
- `key_name` - The name of the index, `PRIMARY` if the index is a primary key index. (String)
- `seq_in_index` - For a primary key index, the position of the column starting from `1`. For a data skipping index: always `1`. (UInt8)
- `column_name` - For a primary key index, the name of the column. For a data skipping index: `''` (empty string), see field "expression". (String)
- `collation` - The sorting of the column in the index: `A` if ascending, `D` if descending, `NULL` if unsorted. (Nullable(String))
- `cardinality` - An estimation of the index cardinality (number of unique values in the index). Currently always 0. (UInt64)
- `sub_part` - Always `NULL` because ClickHouse does not support index prefixes like MySQL. (Nullable(String))
- `packed` - Always `NULL` because ClickHouse does not support packed indexes (like MySQL). (Nullable(String))
- `null` - Currently unused
- `index_type` - The index type, e.g. `PRIMARY`, `MINMAX`, `BLOOM_FILTER` etc. (String)
- `comment` - Additional information about the index, currently always `''` (empty string). (String)
- `index_comment` - `''` (empty string) because indexes in ClickHouse cannot have a `COMMENT` field (like in MySQL). (String)
- `visible` - If the index is visible to the optimizer, always `YES`. (String)
- `expression` - For a data skipping index, the index expression. For a primary key index: `''` (empty string). (String)
**Examples**

View File

@ -119,6 +119,18 @@ The compiled expression cache is enabled/disabled with the query/user/profile-le
Clears the [query cache](../../operations/query-cache.md).
## DROP FORMAT SCHEMA CACHE {#system-drop-schema-format}
Clears cache for schemas loaded from [format_schema_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path).
Supported formats:
- Protobuf
```sql
SYSTEM DROP FORMAT SCHEMA CACHE [FOR Protobuf]
```
## FLUSH LOGS
Flushes buffered log messages to system tables, e.g. system.query_log. Mainly useful for debugging since most system tables have a default flush interval of 7.5 seconds.

View File

@ -20,7 +20,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2],
...
) ENGINE = ReplacingMergeTree([ver])
) ENGINE = ReplacingMergeTree([ver [, is_deleted]])
[PARTITION BY expr]
[ORDER BY expr]
[SAMPLE BY expr]
@ -86,6 +86,59 @@ SELECT * FROM mySecondReplacingMT FINAL;
│ 1 │ first │ 2020-01-01 01:01:01 │
└─────┴─────────┴─────────────────────┘
```
### is_deleted
`is_deleted` — Имя столбца, который используется во время слияния для обозначения того, нужно ли отображать строку или она подлежит удалению; `1` - для удаления строки, `0` - для отображения строки.
Тип данных столбца — `UInt8`.
:::note
`is_deleted` может быть использован, если `ver` используется.
Строка удаляется в следующих случаях:
- при использовании инструкции `OPTIMIZE ... FINAL CLEANUP`
- при использовании инструкции `OPTIMIZE ... FINAL`
- параметр движка `clean_deleted_rows` установлен в значение `Always` (по умолчанию - `Never`)
- есть новые версии строки
Не рекомендуется выполнять `FINAL CLEANUP` или использовать параметр движка `clean_deleted_rows` со значением `Always`, это может привести к неожиданным результатам, например удаленные строки могут вновь появиться.
Вне зависимости от производимых изменений над данными, версия должна увеличиваться. Если у двух строк одна и та же версия, то остается только последняя вставленная строка.
:::
Пример:
```sql
-- with ver and is_deleted
CREATE OR REPLACE TABLE myThirdReplacingMT
(
`key` Int64,
`someCol` String,
`eventTime` DateTime,
`is_deleted` UInt8
)
ENGINE = ReplacingMergeTree(eventTime, is_deleted)
ORDER BY key;
INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 01:01:01', 0);
INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 01:01:01', 1);
select * from myThirdReplacingMT final;
0 rows in set. Elapsed: 0.003 sec.
-- delete rows with is_deleted
OPTIMIZE TABLE myThirdReplacingMT FINAL CLEANUP;
INSERT INTO myThirdReplacingMT Values (1, 'first', '2020-01-01 00:00:00', 0);
select * from myThirdReplacingMT final;
┌─key─┬─someCol─┬───────────eventTime─┬─is_deleted─┐
│ 1 │ first │ 2020-01-01 00:00:00 │ 0 │
└─────┴─────────┴─────────────────────┴────────────┘
```
## Секции запроса

View File

@ -50,7 +50,7 @@ Connection: Close
Content-Type: text/tab-separated-values; charset=UTF-8
X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal
X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f
X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
1
```
@ -267,9 +267,9 @@ $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:812
Прогресс выполнения запроса можно отслеживать с помощью заголовков ответа `X-ClickHouse-Progress`. Для этого включите [send_progress_in_http_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Пример последовательности заголовков:
``` text
X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","elapsed_ns":"662334","peak_memory_usage":"4371480"}
X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","elapsed_ns":"992334","peak_memory_usage":"13621616"}
X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","elapsed_ns":"1232334","peak_memory_usage":"23155600"}
X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","elapsed_ns":"662334"}
X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","elapsed_ns":"992334"}
X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","elapsed_ns":"1232334"}
```
Возможные поля заголовка:
@ -530,7 +530,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi'
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
* Connection #0 to host localhost left intact
Say Hi!%
@ -570,7 +570,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler'
< Content-Type: text/plain; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
* Connection #0 to host localhost left intact
<html ng-app="SMI2"><head><base href="http://ui.tabix.io/"></head><body><div ui-view="" class="content-ui"></div><script src="http://loader.tabix.io/master.js"></script></body></html>%
@ -622,7 +622,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler'
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
<html><body>Absolute Path File</body></html>
* Connection #0 to host localhost left intact
@ -641,7 +641,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler'
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
<html><body>Relative Path File</body></html>
* Connection #0 to host localhost left intact

View File

@ -37,6 +37,8 @@ ClickHouse Inc. не поддерживает перечисленные ниж
- [nestjs-clickhouse](https://github.com/depyronick/nestjs-clickhouse)
- [clickhouse-client](https://github.com/depyronick/clickhouse-client)
- [node-clickhouse-orm](https://github.com/zimv/node-clickhouse-orm)
- [clickhouse-ts](https://github.com/bytadaniel/clickhouse-ts)
- [clickcache](https://github.com/bytadaniel/clickcache)
- Perl
- [perl-DBD-ClickHouse](https://github.com/elcamlost/perl-DBD-ClickHouse)
- [HTTP-ClickHouse](https://metacpan.org/release/HTTP-ClickHouse)

View File

@ -61,7 +61,7 @@ PARTITION BY expr
- limit_size: 必需的。本地缓存文件的最大大小(单位为字节)。
- bytes_read_before_flush: 从远程文件系统下载文件时刷新到本地文件系统前的控制字节数。缺省值为1MB。
当ClickHouse为远程文件系统启用了本地缓存时用户仍然可以选择不使用缓存并在查询中设置`use_local_cache_for_remote_fs = 0 `, `use_local_cache_for_remote_fs` 默认为 `false`。
当ClickHouse为远程文件系统启用了本地缓存时用户仍然可以选择不使用缓存并在查询中设置 `use_local_cache_for_remote_storage = 0`, `use_local_cache_for_remote_storage` 默认为 `1`。
### 查询 ORC 输入格式的Hive 表

View File

@ -53,7 +53,7 @@ Connection: Close
Content-Type: text/tab-separated-values; charset=UTF-8
X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal
X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f
X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
1
```
@ -262,9 +262,9 @@ $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:812
您可以在`X-ClickHouse-Progress`响应头中收到查询进度的信息。为此,启用[Http Header携带进度](../operations/settings/settings.md#settings-send_progress_in_http_headers)。示例:
``` text
X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","elapsed_ns":"662334","peak_memory_usage":"4371480"}
X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","elapsed_ns":"992334","peak_memory_usage":"13621616"}
X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","elapsed_ns":"1232334","peak_memory_usage":"23155600"}
X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","elapsed_ns":"662334"}
X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","elapsed_ns":"992334"}
X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","elapsed_ns":"1232334"}
```
显示字段信息:
@ -363,7 +363,7 @@ $ curl -v 'http://localhost:8123/predefined_query'
< X-ClickHouse-Format: Template
< X-ClickHouse-Timezone: Asia/Shanghai
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
# HELP "Query" "Number of executing queries"
# TYPE "Query" counter
@ -521,7 +521,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi'
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
* Connection #0 to host localhost left intact
Say Hi!%
@ -561,7 +561,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler'
< Content-Type: text/plain; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
* Connection #0 to host localhost left intact
<html ng-app="SMI2"><head><base href="http://ui.tabix.io/"></head><body><div ui-view="" class="content-ui"></div><script src="http://loader.tabix.io/master.js"></script></body></html>%
@ -613,7 +613,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler'
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
<html><body>Absolute Path File</body></html>
* Connection #0 to host localhost left intact
@ -632,7 +632,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler'
< Content-Type: text/html; charset=UTF-8
< Transfer-Encoding: chunked
< Keep-Alive: timeout=3
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"}
< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"}
<
<html><body>Relative Path File</body></html>
* Connection #0 to host localhost left intact

View File

@ -39,6 +39,8 @@ Yandex**没有**维护下面列出的库,也没有做过任何广泛的测试
- [nestjs-clickhouse](https://github.com/depyronick/nestjs-clickhouse)
- [clickhouse-client](https://github.com/depyronick/clickhouse-client)
- [node-clickhouse-orm](https://github.com/zimv/node-clickhouse-orm)
- [clickhouse-ts](https://github.com/bytadaniel/clickhouse-ts)
- [clickcache](https://github.com/bytadaniel/clickcache)
- Perl
- [perl-DBD-ClickHouse](https://github.com/elcamlost/perl-DBD-ClickHouse)
- [HTTP-ClickHouse](https://metacpan.org/release/HTTP-ClickHouse)

View File

@ -84,8 +84,8 @@ function deb2tgz {
FILE=$1
PKG_NAME=${FILE##*/}; PKG_NAME=${PKG_NAME%%_*}
PKG_DIR="$PKG_NAME-$CLICKHOUSE_VERSION_STRING"
PKG_PATH="$OUTPUT_DIR/$PKG_NAME-$CLICKHOUSE_VERSION_STRING"
TARBALL="$OUTPUT_DIR/$PKG_NAME-$CLICKHOUSE_VERSION_STRING-$DEB_ARCH.tgz"
PKG_PATH="$OUTPUT_DIR/$PKG_DIR"
TARBALL="$OUTPUT_DIR/$PKG_DIR-$DEB_ARCH.tgz"
rm -rf "$PKG_PATH"
dpkg-deb -R "$FILE" "$PKG_PATH"
mkdir -p "$PKG_PATH/install"

View File

@ -1,6 +1,13 @@
# package sources should be placed in ${PWD}/root
# nfpm should run from the same directory with a config
name: "clickhouse-client"
description: |
Client binary for ClickHouse
ClickHouse is a column-oriented database management system.
that allows generating analytical data reports in real time.
This package provides clickhouse-client, clickhouse-local and clickhouse-benchmark.
# Common packages config
arch: "${DEB_ARCH}" # amd64, arm64
platform: "linux"
version: "${CLICKHOUSE_VERSION_STRING}"
@ -9,19 +16,17 @@ homepage: "https://clickhouse.com"
license: "Apache"
section: "database"
priority: "optional"
maintainer: "ClickHouse Dev Team <packages+linux@clickhouse.com>"
deb:
fields:
Source: clickhouse
# Package specific content
replaces:
- clickhouse-compressor
conflicts:
- clickhouse-compressor
maintainer: "ClickHouse Dev Team <packages+linux@clickhouse.com>"
description: |
Client binary for ClickHouse
ClickHouse is a column-oriented database management system.
that allows generating analytical data reports in real time.
This package provides clickhouse-client, clickhouse-local and clickhouse-benchmark.
overrides:
deb:
depends:
@ -30,10 +35,6 @@ overrides:
depends:
- clickhouse-common-static = ${CLICKHOUSE_VERSION_STRING}
deb:
fields:
Source: clickhouse
contents:
- src: root/etc/clickhouse-client/config.xml
dst: /etc/clickhouse-client/config.xml

View File

@ -1,6 +1,13 @@
# package sources should be placed in ${PWD}/root
# nfpm should run from the same directory with a config
name: "clickhouse-common-static-dbg"
description: |
debugging symbols for clickhouse-common-static
This package contains the debugging symbols for clickhouse-common.
#
# Common packages config
arch: "${DEB_ARCH}" # amd64, arm64
platform: "linux"
version: "${CLICKHOUSE_VERSION_STRING}"
@ -9,21 +16,17 @@ homepage: "https://clickhouse.com"
license: "Apache"
section: "database"
priority: "optional"
maintainer: "ClickHouse Dev Team <packages+linux@clickhouse.com>"
deb:
fields:
Source: clickhouse
# Package specific content
replaces:
- clickhouse-common-dbg
conflicts:
- clickhouse-common-dbg
maintainer: "ClickHouse Dev Team <packages+linux@clickhouse.com>"
description: |
debugging symbols for clickhouse-common-static
This package contains the debugging symbols for clickhouse-common.
deb:
fields:
Source: clickhouse
contents:
- src: root/usr/lib/debug/usr/bin/clickhouse.debug
dst: /usr/lib/debug/usr/bin/clickhouse.debug

View File

@ -1,6 +1,13 @@
# package sources should be placed in ${PWD}/root
# nfpm should run from the same directory with a config
name: "clickhouse-common-static"
description: |
Common files for ClickHouse
ClickHouse is a column-oriented database management system
that allows generating analytical data reports in real time.
This package provides common files for both clickhouse server and client
# Common packages config
arch: "${DEB_ARCH}" # amd64, arm64
platform: "linux"
version: "${CLICKHOUSE_VERSION_STRING}"
@ -9,7 +16,12 @@ homepage: "https://clickhouse.com"
license: "Apache"
section: "database"
priority: "optional"
maintainer: "ClickHouse Dev Team <packages+linux@clickhouse.com>"
deb:
fields:
Source: clickhouse
# Package specific content
replaces:
- clickhouse-common
- clickhouse-server-base
@ -19,17 +31,6 @@ provides:
suggests:
- clickhouse-common-static-dbg
maintainer: "ClickHouse Dev Team <packages+linux@clickhouse.com>"
description: |
Common files for ClickHouse
ClickHouse is a column-oriented database management system
that allows generating analytical data reports in real time.
This package provides common files for both clickhouse server and client
deb:
fields:
Source: clickhouse
contents:
- src: root/usr/bin/clickhouse
dst: /usr/bin/clickhouse

View File

@ -1,6 +1,13 @@
# package sources should be placed in ${PWD}/root
# nfpm should run from the same directory with a config
name: "clickhouse-keeper-dbg"
description: |
debugging symbols for clickhouse-keeper
This package contains the debugging symbols for clickhouse-keeper.
#
# Common packages config
arch: "${DEB_ARCH}" # amd64, arm64
platform: "linux"
version: "${CLICKHOUSE_VERSION_STRING}"
@ -10,14 +17,11 @@ license: "Apache"
section: "database"
priority: "optional"
maintainer: "ClickHouse Dev Team <packages+linux@clickhouse.com>"
description: |
debugging symbols for clickhouse-keeper
This package contains the debugging symbols for clickhouse-keeper.
deb:
fields:
Source: clickhouse
# Package specific content
contents:
- src: root/usr/lib/debug/usr/bin/clickhouse-keeper.debug
dst: /usr/lib/debug/usr/bin/clickhouse-keeper.debug

View File

@ -3,12 +3,12 @@ set -e
# set -x
PROGRAM=clickhouse-keeper
KEEPER_USER=${KEEPER_USER:=clickhouse}
KEEPER_GROUP=${KEEPER_GROUP:=clickhouse}
KEEPER_USER=${KEEPER_USER:-clickhouse}
KEEPER_GROUP=${KEEPER_GROUP:-clickhouse}
# Please note that we don't support paths with whitespaces. This is rather ignorant.
KEEPER_CONFDIR=${KEEPER_CONFDIR:=/etc/$PROGRAM}
KEEPER_DATADIR=${KEEPER_DATADIR:=/var/lib/clickhouse}
KEEPER_LOGDIR=${KEEPER_LOGDIR:=/var/log/$PROGRAM}
KEEPER_CONFDIR=${KEEPER_CONFDIR:-/etc/$PROGRAM}
KEEPER_DATADIR=${KEEPER_DATADIR:-/var/lib/clickhouse}
KEEPER_LOGDIR=${KEEPER_LOGDIR:-/var/log/$PROGRAM}
[ -f /usr/share/debconf/confmodule ] && . /usr/share/debconf/confmodule
[ -f /etc/default/clickhouse-keeper ] && . /etc/default/clickhouse-keeper

View File

@ -1,6 +1,13 @@
# package sources should be placed in ${PWD}/root
# nfpm should run from the same directory with a config
name: "clickhouse-keeper"
description: |
Static clickhouse-keeper binary
A stand-alone clickhouse-keeper package
#
# Common packages config
arch: "${DEB_ARCH}" # amd64, arm64
platform: "linux"
version: "${CLICKHOUSE_VERSION_STRING}"
@ -9,29 +16,25 @@ homepage: "https://clickhouse.com"
license: "Apache"
section: "database"
priority: "optional"
maintainer: "ClickHouse Dev Team <packages+linux@clickhouse.com>"
deb:
fields:
Source: clickhouse
# Package specific content
conflicts:
- clickhouse-server
suggests:
- clickhouse-keeper-dbg
maintainer: "ClickHouse Dev Team <packages+linux@clickhouse.com>"
description: |
Static clickhouse-keeper binary
A stand-alone clickhouse-keeper package
deb:
fields:
Source: clickhouse
contents:
- src: root/etc/clickhouse-keeper/keeper_config.xml
dst: /etc/clickhouse-keeper/keeper_config.xml
type: config|noreplace
- src: root/usr/bin/clickhouse-keeper
dst: /usr/bin/clickhouse-keeper
- src: clickhouse-keeper.service
dst: /lib/systemd/system/clickhouse-keeper.service
- src: root/usr/bin/clickhouse-keeper
dst: /usr/bin/clickhouse-keeper
- src: clickhouse-keeper
dst: /usr/bin/clickhouse-keeper-client
type: symlink

View File

@ -3,16 +3,21 @@ set -e
# set -x
PROGRAM=clickhouse-server
CLICKHOUSE_USER=${CLICKHOUSE_USER:=clickhouse}
CLICKHOUSE_GROUP=${CLICKHOUSE_GROUP:=${CLICKHOUSE_USER}}
CLICKHOUSE_USER=${CLICKHOUSE_USER:-clickhouse}
CLICKHOUSE_GROUP=${CLICKHOUSE_GROUP:-${CLICKHOUSE_USER}}
# Please note that we don't support paths with whitespaces. This is rather ignorant.
CLICKHOUSE_CONFDIR=${CLICKHOUSE_CONFDIR:=/etc/clickhouse-server}
CLICKHOUSE_DATADIR=${CLICKHOUSE_DATADIR:=/var/lib/clickhouse}
CLICKHOUSE_LOGDIR=${CLICKHOUSE_LOGDIR:=/var/log/clickhouse-server}
CLICKHOUSE_BINDIR=${CLICKHOUSE_BINDIR:=/usr/bin}
CLICKHOUSE_GENERIC_PROGRAM=${CLICKHOUSE_GENERIC_PROGRAM:=clickhouse}
CLICKHOUSE_CONFDIR=${CLICKHOUSE_CONFDIR:-/etc/clickhouse-server}
CLICKHOUSE_DATADIR=${CLICKHOUSE_DATADIR:-/var/lib/clickhouse}
CLICKHOUSE_LOGDIR=${CLICKHOUSE_LOGDIR:-/var/log/clickhouse-server}
CLICKHOUSE_BINDIR=${CLICKHOUSE_BINDIR:-/usr/bin}
CLICKHOUSE_GENERIC_PROGRAM=${CLICKHOUSE_GENERIC_PROGRAM:-clickhouse}
CLICKHOUSE_PIDDIR=/var/run/$PROGRAM
# Provide clickhouse-keeper
KEEPER_CONFDIR=${KEEPER_CONFDIR:-/etc/clickhouse-keeper}
KEEPER_DATADIR=${KEEPER_DATADIR:-/var/lib/clickhouse}
KEEPER_LOGDIR=${KEEPER_LOGDIR:-/var/log/clickhouse-keeper}
[ -f /usr/share/debconf/confmodule ] && . /usr/share/debconf/confmodule
[ -f /etc/default/clickhouse ] && . /etc/default/clickhouse
@ -54,4 +59,20 @@ if [ "$1" = configure ] || [ -n "$not_deb_os" ]; then
fi
done
fi
# Setup clickhouse-keeper directories
chown -R "${CLICKHOUSE_USER}:${CLICKHOUSE_GROUP}" "${KEEPER_CONFDIR}"
chmod 0755 "${KEEPER_CONFDIR}"
if ! [ -d "${KEEPER_DATADIR}" ]; then
mkdir -p "${KEEPER_DATADIR}"
chown -R "${CLICKHOUSE_USER}:${CLICKHOUSE_GROUP}" "${KEEPER_DATADIR}"
chmod 0700 "${KEEPER_DATADIR}"
fi
if ! [ -d "${KEEPER_LOGDIR}" ]; then
mkdir -p "${KEEPER_LOGDIR}"
chown -R "${CLICKHOUSE_USER}:${CLICKHOUSE_GROUP}" "${KEEPER_LOGDIR}"
chmod 0770 "${KEEPER_LOGDIR}"
fi
fi

View File

@ -1,6 +1,13 @@
# package sources should be placed in ${PWD}/root
# nfpm should run from the same directory with a config
name: "clickhouse-server"
description: |
Server binary for ClickHouse
ClickHouse is a column-oriented database management system
that allows generating analytical data reports in real time.
This package provides clickhouse common configuration files
# Common packages config
arch: "${DEB_ARCH}" # amd64, arm64
platform: "linux"
version: "${CLICKHOUSE_VERSION_STRING}"
@ -9,24 +16,21 @@ homepage: "https://clickhouse.com"
license: "Apache"
section: "database"
priority: "optional"
maintainer: "ClickHouse Dev Team <packages+linux@clickhouse.com>"
deb:
fields:
Source: clickhouse
conflicts:
- clickhouse-keeper
# Package specific content
replaces:
- clickhouse-server-common
- clickhouse-server-base
provides:
- clickhouse-keeper
- clickhouse-server-common
recommends:
- libcap2-bin
maintainer: "ClickHouse Dev Team <packages+linux@clickhouse.com>"
description: |
Server binary for ClickHouse
ClickHouse is a column-oriented database management system
that allows generating analytical data reports in real time.
This package provides clickhouse common configuration files
overrides:
deb:
depends:
@ -35,10 +39,6 @@ overrides:
depends:
- clickhouse-common-static = ${CLICKHOUSE_VERSION_STRING}
deb:
fields:
Source: clickhouse
contents:
- src: root/etc/clickhouse-server/config.xml
dst: /etc/clickhouse-server/config.xml
@ -52,16 +52,25 @@ contents:
dst: /lib/systemd/system/clickhouse-server.service
- src: root/usr/bin/clickhouse-copier
dst: /usr/bin/clickhouse-copier
- src: root/usr/bin/clickhouse-report
dst: /usr/bin/clickhouse-report
- src: root/usr/bin/clickhouse-server
dst: /usr/bin/clickhouse-server
# clickhouse-keeper part
- src: root/etc/clickhouse-keeper/keeper_config.xml
dst: /etc/clickhouse-keeper/keeper_config.xml
type: config|noreplace
- src: clickhouse-keeper.service
dst: /lib/systemd/system/clickhouse-keeper.service
- src: clickhouse
dst: /usr/bin/clickhouse-keeper
type: symlink
- src: clickhouse
dst: /usr/bin/clickhouse-keeper-client
type: symlink
- src: root/usr/bin/clickhouse-report
dst: /usr/bin/clickhouse-report
- src: root/usr/bin/clickhouse-server
dst: /usr/bin/clickhouse-server
- src: clickhouse
dst: /usr/bin/clickhouse-keeper-converter
type: symlink
# docs
- src: ../AUTHORS
dst: /usr/share/doc/clickhouse-server/AUTHORS

View File

@ -706,6 +706,17 @@ bool Client::processWithFuzzing(const String & full_query)
return true;
}
// Kusto is not a subject for fuzzing (yet)
if (global_context->getSettingsRef().dialect == DB::Dialect::kusto)
{
return true;
}
if (auto *q = orig_ast->as<ASTSetQuery>())
{
if (auto *setDialect = q->changes.tryGet("dialect"); setDialect && setDialect->safeGet<String>() == "kusto")
return true;
}
// Don't repeat:
// - INSERT -- Because the tables may grow too big.
// - CREATE -- Because first we run the unmodified query, it will succeed,
@ -1253,6 +1264,15 @@ void Client::processConfig()
global_context->setQueryKindInitial();
global_context->setQuotaClientKey(config().getString("quota_key", ""));
global_context->setQueryKind(query_kind);
if (is_multiquery && !global_context->getSettingsRef().input_format_values_allow_data_after_semicolon.changed)
{
Settings settings = global_context->getSettings();
settings.input_format_values_allow_data_after_semicolon = true;
/// Do not send it to the server
settings.input_format_values_allow_data_after_semicolon.changed = false;
global_context->setSettings(settings);
}
}

View File

@ -269,7 +269,7 @@ ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std
res.hostname_difference = std::numeric_limits<size_t>::max();
for (const auto & replica : replicas)
{
size_t difference = getHostNameDifference(local_hostname, replica.host_name);
size_t difference = getHostNamePrefixDistance(local_hostname, replica.host_name);
res.hostname_difference = std::min(difference, res.hostname_difference);
}

View File

@ -324,8 +324,8 @@ void FindBigFamily::execute(const ASTKeeperQuery * query, KeeperClient * client)
queue.pop();
auto children = client->zookeeper->getChildren(next_path);
std::transform(children.cbegin(), children.cend(), children.begin(), [&](const String & child) { return next_path / child; });
for (auto & child : children)
child = next_path / child;
auto response = client->zookeeper->get(children);
for (size_t i = 0; i < response.size(); ++i)
@ -475,4 +475,45 @@ void FourLetterWordCommand::execute(const ASTKeeperQuery * query, KeeperClient *
std::cout << client->executeFourLetterCommand(query->args[0].safeGet<String>()) << "\n";
}
bool GetAllChildrenNumberCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
{
String path;
if (!parseKeeperPath(pos, expected, path))
path = ".";
node->args.push_back(std::move(path));
return true;
}
void GetAllChildrenNumberCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
{
auto path = client->getAbsolutePath(query->args[0].safeGet<String>());
std::queue<fs::path> queue;
queue.push(path);
Coordination::Stat stat;
client->zookeeper->get(path, &stat);
int totalNumChildren = stat.numChildren;
while (!queue.empty())
{
auto next_path = queue.front();
queue.pop();
auto children = client->zookeeper->getChildren(next_path);
for (auto & child : children)
child = next_path / child;
auto response = client->zookeeper->get(children);
for (size_t i = 0; i < response.size(); ++i)
{
totalNumChildren += response[i].stat.numChildren;
queue.push(children[i]);
}
}
std::cout << totalNumChildren << "\n";
}
}

View File

@ -238,4 +238,18 @@ class FourLetterWordCommand : public IKeeperClientCommand
String getHelpMessage() const override { return "{} <command> -- Executes four-letter-word command"; }
};
class GetAllChildrenNumberCommand : public IKeeperClientCommand
{
String getName() const override { return "get_all_children_number"; }
bool parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const override;
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
String getHelpMessage() const override
{
return "{} [path] -- Get all numbers of children nodes under a specific path";
}
};
}

View File

@ -2,6 +2,7 @@
#include "Commands.h"
#include <Client/ReplxxLineReader.h>
#include <Client/ClientBase.h>
#include "Common/VersionNumber.h"
#include <Common/Config/ConfigProcessor.h>
#include <Common/EventNotifier.h>
#include <Common/filesystemHelpers.h>
@ -206,6 +207,7 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */)
std::make_shared<SyncCommand>(),
std::make_shared<HelpCommand>(),
std::make_shared<FourLetterWordCommand>(),
std::make_shared<GetAllChildrenNumberCommand>(),
});
String home_path;

View File

@ -783,6 +783,15 @@ void LocalServer::processConfig()
global_context->setQueryKindInitial();
global_context->setQueryKind(query_kind);
if (is_multiquery && !global_context->getSettingsRef().input_format_values_allow_data_after_semicolon.changed)
{
Settings settings = global_context->getSettings();
settings.input_format_values_allow_data_after_semicolon = true;
/// Do not send it to the server
settings.input_format_values_allow_data_after_semicolon.changed = false;
global_context->setSettings(settings);
}
}

View File

@ -1474,7 +1474,7 @@ try
{
std::lock_guard lock(servers_lock);
/// We should start interserver communications before (and more imporant shutdown after) tables.
/// We should start interserver communications before (and more important shutdown after) tables.
/// Because server can wait for a long-running queries (for example in tcp_handler) after interserver handler was already shut down.
/// In this case we will have replicated tables which are unable to send any parts to other replicas, but still can
/// communicate with zookeeper, execute merges, etc.

View File

@ -420,6 +420,10 @@
color: var(--auth-error-color);
}
#charts > div:only-child .display-only-if-more-than-one-chart {
display: none;
}
/* Source: https://cdn.jsdelivr.net/npm/uplot@1.6.21/dist/uPlot.min.css
* It is copy-pasted to lower the number of requests.
*/
@ -931,6 +935,9 @@ function insertChart(i) {
saveState();
});
move.classList.add('display-only-if-more-than-one-chart');
maximize.classList.add('display-only-if-more-than-one-chart');
edit_buttons.appendChild(move);
edit_buttons.appendChild(maximize);
edit_buttons.appendChild(edit);

View File

@ -9,6 +9,7 @@ profiles:
# random - choose random replica from set of replicas with minimum number of errors
# nearest_hostname - from set of replicas with minimum number of errors, choose replica
# with minimum number of different symbols between replica's hostname and local hostname (Hamming distance).
# hostname_levenshtein_distance - just the same with nearest_hostname but calculate the difference by Levenshtein distance.
# in_order - first live replica is chosen in specified order.
# first_or_random - if first replica one has higher number of errors, pick a random one from replicas with minimum number of errors.
load_balancing: random

View File

@ -13,6 +13,13 @@ if (OS_FREEBSD)
message(STATUS "skim is disabled for FreeBSD")
return()
endif()
if (SANITIZE STREQUAL "thread")
# Rust does not supports Thread Sanitizer [1]
#
# [1]: https://doc.rust-lang.org/beta/unstable-book/compiler-flags/sanitizer.html#threadsanitizer
message(STATUS "skim is disabled under Thread Sanitizer")
return()
endif()
clickhouse_import_crate(MANIFEST_PATH Cargo.toml)

View File

@ -155,6 +155,7 @@ enum class AccessType
M(SYSTEM_DROP_FILESYSTEM_CACHE, "SYSTEM DROP FILESYSTEM CACHE, DROP FILESYSTEM CACHE", GLOBAL, SYSTEM_DROP_CACHE) \
M(SYSTEM_SYNC_FILESYSTEM_CACHE, "SYSTEM REPAIR FILESYSTEM CACHE, REPAIR FILESYSTEM CACHE, SYNC FILESYSTEM CACHE", GLOBAL, SYSTEM) \
M(SYSTEM_DROP_SCHEMA_CACHE, "SYSTEM DROP SCHEMA CACHE, DROP SCHEMA CACHE", GLOBAL, SYSTEM_DROP_CACHE) \
M(SYSTEM_DROP_FORMAT_SCHEMA_CACHE, "SYSTEM DROP FORMAT SCHEMA CACHE, DROP FORMAT SCHEMA CACHE", GLOBAL, SYSTEM_DROP_CACHE) \
M(SYSTEM_DROP_S3_CLIENT_CACHE, "SYSTEM DROP S3 CLIENT, DROP S3 CLIENT CACHE", GLOBAL, SYSTEM_DROP_CACHE) \
M(SYSTEM_DROP_CACHE, "DROP CACHE", GROUP, SYSTEM) \
M(SYSTEM_RELOAD_CONFIG, "RELOAD CONFIG", GLOBAL, SYSTEM_RELOAD) \

View File

@ -61,7 +61,7 @@ private: // IAccessStorage implementations.
bool areLDAPCredentialsValidNoLock(const User & user, const Credentials & credentials,
const ExternalAuthenticators & external_authenticators, LDAPClient::SearchResultsList & role_search_results) const;
mutable std::recursive_mutex mutex;
mutable std::recursive_mutex mutex; // Note: Reentrace possible by internal role lookup via access_control
AccessControl & access_control;
String ldap_server_name;
LDAPClient::RoleSearchParamsList role_search_params;

View File

@ -44,12 +44,12 @@ private:
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
bool insertNoLock(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) TSA_REQUIRES(mutex);
bool removeNoLock(const UUID & id, bool throw_if_not_exists) TSA_REQUIRES(mutex);
bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) TSA_REQUIRES(mutex);
bool insertNoLock(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
bool removeNoLock(const UUID & id, bool throw_if_not_exists);
bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists);
void removeAllExceptNoLock(const std::vector<UUID> & ids_to_keep) TSA_REQUIRES(mutex);
void removeAllExceptNoLock(const boost::container::flat_set<UUID> & ids_to_keep) TSA_REQUIRES(mutex);
void removeAllExceptNoLock(const std::vector<UUID> & ids_to_keep);
void removeAllExceptNoLock(const boost::container::flat_set<UUID> & ids_to_keep);
struct Entry
{
@ -57,9 +57,9 @@ private:
AccessEntityPtr entity;
};
mutable std::mutex mutex;
std::unordered_map<UUID, Entry> entries_by_id TSA_GUARDED_BY(mutex); /// We want to search entries both by ID and by the pair of name and type.
std::unordered_map<String, Entry *> entries_by_name_and_type[static_cast<size_t>(AccessEntityType::MAX)] TSA_GUARDED_BY(mutex);
mutable std::recursive_mutex mutex; // Note: Reentrace possible via LDAPAccessStorage
std::unordered_map<UUID, Entry> entries_by_id; /// We want to search entries both by ID and by the pair of name and type.
std::unordered_map<String, Entry *> entries_by_name_and_type[static_cast<size_t>(AccessEntityType::MAX)];
AccessChangesNotifier & changes_notifier;
const bool backup_allowed = false;
};

View File

@ -1264,7 +1264,8 @@ private:
size_t identifier_bind_size,
const QueryTreeNodePtr & compound_expression,
String compound_expression_source,
IdentifierResolveScope & scope);
IdentifierResolveScope & scope,
bool can_be_not_found = false);
QueryTreeNodePtr tryResolveIdentifierFromExpressionArguments(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope);
@ -1313,6 +1314,14 @@ private:
IdentifierResolveScope & scope,
IdentifierResolveSettings identifier_resolve_settings = {});
QueryTreeNodePtr tryResolveIdentifierFromStorage(
const Identifier & identifier,
const QueryTreeNodePtr & table_expression_node,
const TableExpressionData & table_expression_data,
IdentifierResolveScope & scope,
size_t identifier_column_qualifier_parts,
bool can_be_not_found = false);
/// Resolve query tree nodes functions
void qualifyColumnNodesWithProjectionNames(const QueryTreeNodes & column_nodes,
@ -2395,11 +2404,13 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con
}
/// Resolve identifier from compound expression
/// If identifier cannot be resolved throw exception or return nullptr if can_be_not_found is true
QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromCompoundExpression(const Identifier & expression_identifier,
size_t identifier_bind_size,
const QueryTreeNodePtr & compound_expression,
String compound_expression_source,
IdentifierResolveScope & scope)
IdentifierResolveScope & scope,
bool can_be_not_found)
{
Identifier compound_expression_identifier;
for (size_t i = 0; i < identifier_bind_size; ++i)
@ -2412,6 +2423,23 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromCompoundExpression(const
if (!expression_type->hasSubcolumn(nested_path.getFullName()))
{
if (auto * column = compound_expression->as<ColumnNode>())
{
const DataTypePtr & column_type = column->getColumn().getTypeInStorage();
if (column_type->getTypeId() == TypeIndex::Object)
{
const auto * object_type = checkAndGetDataType<DataTypeObject>(column_type.get());
if (object_type->getSchemaFormat() == "json" && object_type->hasNullableSubcolumns())
{
QueryTreeNodePtr constant_node_null = std::make_shared<ConstantNode>(Field());
return constant_node_null;
}
}
}
if (can_be_not_found)
return {};
std::unordered_set<Identifier> valid_identifiers;
collectCompoundExpressionValidIdentifiersForTypoCorrection(expression_identifier,
expression_type,
@ -2427,20 +2455,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromCompoundExpression(const
compound_expression_from_error_message += compound_expression_source;
}
if (auto * column = compound_expression->as<ColumnNode>())
{
const DataTypePtr & column_type = column->getColumn().getTypeInStorage();
if (column_type->getTypeId() == TypeIndex::Object)
{
const auto * object_type = checkAndGetDataType<DataTypeObject>(column_type.get());
if (object_type->getSchemaFormat() == "json" && object_type->hasNullableSubcolumns())
{
QueryTreeNodePtr constant_node_null = std::make_shared<ConstantNode>(Field());
return constant_node_null;
}
}
}
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
"Identifier {} nested path {} cannot be resolved from type {}{}. In scope {}{}",
expression_identifier,
@ -2796,6 +2810,160 @@ bool QueryAnalyzer::tryBindIdentifierToTableExpressions(const IdentifierLookup &
return can_bind_identifier_to_table_expression;
}
QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromStorage(
const Identifier & identifier,
const QueryTreeNodePtr & table_expression_node,
const TableExpressionData & table_expression_data,
IdentifierResolveScope & scope,
size_t identifier_column_qualifier_parts,
bool can_be_not_found)
{
auto identifier_without_column_qualifier = identifier;
identifier_without_column_qualifier.popFirst(identifier_column_qualifier_parts);
/** Compound identifier cannot be resolved directly from storage if storage is not table.
*
* Example: SELECT test_table.id.value1.value2 FROM test_table;
* In table storage column test_table.id.value1.value2 will exists.
*
* Example: SELECT test_subquery.compound_expression.value FROM (SELECT compound_expression AS value) AS test_subquery;
* Here there is no column with name test_subquery.compound_expression.value, and additional wrap in tuple element is required.
*/
QueryTreeNodePtr result_expression;
bool match_full_identifier = false;
auto it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.getFullName());
if (it != table_expression_data.column_name_to_column_node.end())
{
match_full_identifier = true;
result_expression = it->second;
}
else
{
it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.at(0));
if (it != table_expression_data.column_name_to_column_node.end())
result_expression = it->second;
}
bool clone_is_needed = true;
String table_expression_source = table_expression_data.table_expression_description;
if (!table_expression_data.table_expression_name.empty())
table_expression_source += " with name " + table_expression_data.table_expression_name;
if (result_expression && !match_full_identifier && identifier_without_column_qualifier.isCompound())
{
size_t identifier_bind_size = identifier_column_qualifier_parts + 1;
result_expression = tryResolveIdentifierFromCompoundExpression(identifier,
identifier_bind_size,
result_expression,
table_expression_source,
scope,
can_be_not_found);
if (can_be_not_found && !result_expression)
return {};
clone_is_needed = false;
}
if (!result_expression)
{
QueryTreeNodes nested_column_nodes;
DataTypes nested_types;
Array nested_names_array;
for (const auto & [column_name, _] : table_expression_data.column_names_and_types)
{
Identifier column_name_identifier_without_last_part(column_name);
auto column_name_identifier_last_part = column_name_identifier_without_last_part.getParts().back();
column_name_identifier_without_last_part.popLast();
if (identifier_without_column_qualifier.getFullName() != column_name_identifier_without_last_part.getFullName())
continue;
auto column_node_it = table_expression_data.column_name_to_column_node.find(column_name);
if (column_node_it == table_expression_data.column_name_to_column_node.end())
continue;
const auto & column_node = column_node_it->second;
const auto & column_type = column_node->getColumnType();
const auto * column_type_array = typeid_cast<const DataTypeArray *>(column_type.get());
if (!column_type_array)
continue;
nested_column_nodes.push_back(column_node);
nested_types.push_back(column_type_array->getNestedType());
nested_names_array.push_back(Field(std::move(column_name_identifier_last_part)));
}
if (!nested_types.empty())
{
auto nested_function_node = std::make_shared<FunctionNode>("nested");
auto & nested_function_node_arguments = nested_function_node->getArguments().getNodes();
auto nested_function_names_array_type = std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
auto nested_function_names_constant_node = std::make_shared<ConstantNode>(std::move(nested_names_array),
std::move(nested_function_names_array_type));
nested_function_node_arguments.push_back(std::move(nested_function_names_constant_node));
nested_function_node_arguments.insert(nested_function_node_arguments.end(),
nested_column_nodes.begin(),
nested_column_nodes.end());
auto nested_function = FunctionFactory::instance().get(nested_function_node->getFunctionName(), scope.context);
nested_function_node->resolveAsFunction(nested_function->build(nested_function_node->getArgumentColumns()));
clone_is_needed = false;
result_expression = std::move(nested_function_node);
}
}
if (!result_expression)
{
std::unordered_set<Identifier> valid_identifiers;
collectTableExpressionValidIdentifiersForTypoCorrection(identifier,
table_expression_node,
table_expression_data,
valid_identifiers);
auto hints = collectIdentifierTypoHints(identifier, valid_identifiers);
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Identifier '{}' cannot be resolved from {}. In scope {}{}",
identifier.getFullName(),
table_expression_source,
scope.scope_node->formatASTForErrorMessage(),
getHintsErrorMessageSuffix(hints));
}
if (clone_is_needed)
result_expression = result_expression->clone();
auto qualified_identifier = identifier;
for (size_t i = 0; i < identifier_column_qualifier_parts; ++i)
{
auto qualified_identifier_with_removed_part = qualified_identifier;
qualified_identifier_with_removed_part.popFirst();
if (qualified_identifier_with_removed_part.empty())
break;
IdentifierLookup column_identifier_lookup = {qualified_identifier_with_removed_part, IdentifierLookupContext::EXPRESSION};
if (tryBindIdentifierToAliases(column_identifier_lookup, scope))
break;
if (table_expression_data.should_qualify_columns &&
tryBindIdentifierToTableExpressions(column_identifier_lookup, table_expression_node, scope))
break;
qualified_identifier = std::move(qualified_identifier_with_removed_part);
}
auto qualified_identifier_full_name = qualified_identifier.getFullName();
node_to_projection_name.emplace(result_expression, std::move(qualified_identifier_full_name));
return result_expression;
}
QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const IdentifierLookup & identifier_lookup,
const QueryTreeNodePtr & table_expression_node,
IdentifierResolveScope & scope)
@ -2836,151 +3004,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id
return {};
}
auto resolve_identifier_from_storage_or_throw = [&](size_t identifier_column_qualifier_parts) -> QueryTreeNodePtr
{
auto identifier_without_column_qualifier = identifier;
identifier_without_column_qualifier.popFirst(identifier_column_qualifier_parts);
/** Compound identifier cannot be resolved directly from storage if storage is not table.
*
* Example: SELECT test_table.id.value1.value2 FROM test_table;
* In table storage column test_table.id.value1.value2 will exists.
*
* Example: SELECT test_subquery.compound_expression.value FROM (SELECT compound_expression AS value) AS test_subquery;
* Here there is no column with name test_subquery.compound_expression.value, and additional wrap in tuple element is required.
*/
QueryTreeNodePtr result_expression;
bool match_full_identifier = false;
auto it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.getFullName());
if (it != table_expression_data.column_name_to_column_node.end())
{
match_full_identifier = true;
result_expression = it->second;
}
else
{
it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.at(0));
if (it != table_expression_data.column_name_to_column_node.end())
result_expression = it->second;
}
bool clone_is_needed = true;
String table_expression_source = table_expression_data.table_expression_description;
if (!table_expression_data.table_expression_name.empty())
table_expression_source += " with name " + table_expression_data.table_expression_name;
if (result_expression && !match_full_identifier && identifier_without_column_qualifier.isCompound())
{
size_t identifier_bind_size = identifier_column_qualifier_parts + 1;
result_expression = tryResolveIdentifierFromCompoundExpression(identifier_lookup.identifier,
identifier_bind_size,
result_expression,
table_expression_source,
scope);
clone_is_needed = false;
}
if (!result_expression)
{
QueryTreeNodes nested_column_nodes;
DataTypes nested_types;
Array nested_names_array;
for (auto & [column_name, _] : table_expression_data.column_names_and_types)
{
Identifier column_name_identifier_without_last_part(column_name);
auto column_name_identifier_last_part = column_name_identifier_without_last_part.getParts().back();
column_name_identifier_without_last_part.popLast();
if (identifier_without_column_qualifier.getFullName() != column_name_identifier_without_last_part.getFullName())
continue;
auto column_node_it = table_expression_data.column_name_to_column_node.find(column_name);
if (column_node_it == table_expression_data.column_name_to_column_node.end())
continue;
const auto & column_node = column_node_it->second;
const auto & column_type = column_node->getColumnType();
const auto * column_type_array = typeid_cast<const DataTypeArray *>(column_type.get());
if (!column_type_array)
continue;
nested_column_nodes.push_back(column_node);
nested_types.push_back(column_type_array->getNestedType());
nested_names_array.push_back(Field(std::move(column_name_identifier_last_part)));
}
if (!nested_types.empty())
{
auto nested_function_node = std::make_shared<FunctionNode>("nested");
auto & nested_function_node_arguments = nested_function_node->getArguments().getNodes();
auto nested_function_names_array_type = std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
auto nested_function_names_constant_node = std::make_shared<ConstantNode>(std::move(nested_names_array),
std::move(nested_function_names_array_type));
nested_function_node_arguments.push_back(std::move(nested_function_names_constant_node));
nested_function_node_arguments.insert(nested_function_node_arguments.end(),
nested_column_nodes.begin(),
nested_column_nodes.end());
auto nested_function = FunctionFactory::instance().get(nested_function_node->getFunctionName(), scope.context);
nested_function_node->resolveAsFunction(nested_function->build(nested_function_node->getArgumentColumns()));
clone_is_needed = false;
result_expression = std::move(nested_function_node);
}
}
if (!result_expression)
{
std::unordered_set<Identifier> valid_identifiers;
collectTableExpressionValidIdentifiersForTypoCorrection(identifier,
table_expression_node,
table_expression_data,
valid_identifiers);
auto hints = collectIdentifierTypoHints(identifier, valid_identifiers);
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Identifier '{}' cannot be resolved from {}. In scope {}{}",
identifier.getFullName(),
table_expression_source,
scope.scope_node->formatASTForErrorMessage(),
getHintsErrorMessageSuffix(hints));
}
if (clone_is_needed)
result_expression = result_expression->clone();
auto qualified_identifier = identifier;
for (size_t i = 0; i < identifier_column_qualifier_parts; ++i)
{
auto qualified_identifier_with_removed_part = qualified_identifier;
qualified_identifier_with_removed_part.popFirst();
if (qualified_identifier_with_removed_part.empty())
break;
IdentifierLookup column_identifier_lookup = {qualified_identifier_with_removed_part, IdentifierLookupContext::EXPRESSION};
if (tryBindIdentifierToAliases(column_identifier_lookup, scope))
break;
if (table_expression_data.should_qualify_columns &&
tryBindIdentifierToTableExpressions(column_identifier_lookup, table_expression_node, scope))
break;
qualified_identifier = std::move(qualified_identifier_with_removed_part);
}
auto qualified_identifier_full_name = qualified_identifier.getFullName();
node_to_projection_name.emplace(result_expression, std::move(qualified_identifier_full_name));
return result_expression;
};
/** If identifier first part binds to some column start or table has full identifier name. Then we can try to find whole identifier in table.
* 1. Try to bind identifier first part to column in table, if true get full identifier from table or throw exception.
* 2. Try to bind identifier first part to table name or storage alias, if true remove first part and try to get full identifier from table or throw exception.
@ -2988,24 +3011,35 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id
* 3. Try to bind identifier first parts to database name and table name, if true remove first two parts and try to get full identifier from table or throw exception.
*/
if (table_expression_data.hasFullIdentifierName(IdentifierView(identifier)))
return resolve_identifier_from_storage_or_throw(0 /*identifier_column_qualifier_parts*/);
return tryResolveIdentifierFromStorage(identifier, table_expression_node, table_expression_data, scope, 0 /*identifier_column_qualifier_parts*/);
if (table_expression_data.canBindIdentifier(IdentifierView(identifier)))
return resolve_identifier_from_storage_or_throw(0 /*identifier_column_qualifier_parts*/);
{
/** This check is insufficient to determine whether and identifier can be resolved from table expression.
* A further check will be performed in `tryResolveIdentifierFromStorage` to see if we have such a subcolumn.
* In cases where the subcolumn cannot be found we want to have `nullptr` instead of exception.
* So, we set `can_be_not_found = true` to have an attempt to resolve the identifier from another table expression.
* Example: `SELECT t.t from (SELECT 1 as t) AS a FULL JOIN (SELECT 1 as t) as t ON a.t = t.t;`
* Initially, we will try to resolve t.t from `a` because `t.` is bound to `1 as t`. However, as it is not a nested column, we will need to resolve it from the second table expression.
*/
auto resolved_identifier = tryResolveIdentifierFromStorage(identifier, table_expression_node, table_expression_data, scope, 0 /*identifier_column_qualifier_parts*/, true /*can_be_not_found*/);
if (resolved_identifier)
return resolved_identifier;
}
if (identifier.getPartsSize() == 1)
return {};
const auto & table_name = table_expression_data.table_name;
if ((!table_name.empty() && path_start == table_name) || (table_expression_node->hasAlias() && path_start == table_expression_node->getAlias()))
return resolve_identifier_from_storage_or_throw(1 /*identifier_column_qualifier_parts*/);
return tryResolveIdentifierFromStorage(identifier, table_expression_node, table_expression_data, scope, 1 /*identifier_column_qualifier_parts*/);
if (identifier.getPartsSize() == 2)
return {};
const auto & database_name = table_expression_data.database_name;
if (!database_name.empty() && path_start == database_name && identifier[1] == table_name)
return resolve_identifier_from_storage_or_throw(2 /*identifier_column_qualifier_parts*/);
return tryResolveIdentifierFromStorage(identifier, table_expression_node, table_expression_data, scope, 2 /*identifier_column_qualifier_parts*/);
return {};
}

View File

@ -591,8 +591,8 @@ if (ENABLE_TESTS)
)
target_link_libraries(unit_tests_dbms PRIVATE
ch_contrib::gtest_all
ch_contrib::gmock_all
ch_contrib::gtest
clickhouse_functions
clickhouse_aggregate_functions
clickhouse_parsers

View File

@ -15,7 +15,6 @@
#include <Common/scope_guard_safe.h>
#include <Common/Exception.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/tests/gtest_global_context.h>
#include <Common/typeid_cast.h>
#include <Common/UTF8Helpers.h>
#include <Common/TerminalSize.h>
@ -47,6 +46,7 @@
#include <Parsers/ASTFunction.h>
#include <Parsers/Kusto/ParserKQLStatement.h>
#include <Parsers/PRQL/ParserPRQLQuery.h>
#include <Parsers/Kusto/parseKQLQuery.h>
#include <Processors/Formats/Impl/NullFormat.h>
#include <Processors/Formats/IInputFormat.h>
@ -349,7 +349,10 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu
if (is_interactive || ignore_error)
{
String message;
res = tryParseQuery(*parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth);
if (dialect == Dialect::kusto)
res = tryParseKQLQuery(*parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth);
else
res = tryParseQuery(*parser, pos, end, message, true, "", allow_multi_statements, max_length, settings.max_parser_depth);
if (!res)
{
@ -359,7 +362,10 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu
}
else
{
res = parseQueryAndMovePosition(*parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth);
if (dialect == Dialect::kusto)
res = parseKQLQueryAndMovePosition(*parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth);
else
res = parseQueryAndMovePosition(*parser, pos, end, "", allow_multi_statements, max_length, settings.max_parser_depth);
}
if (is_interactive)
@ -2013,9 +2019,6 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
{
bool echo_query = echo_queries;
/// Test tags are started with "--" so they are interpreted as comments anyway.
/// But if the echo is enabled we have to remove the test tags from `all_queries_text`
/// because we don't want test tags to be echoed.
{
/// disable logs if expects errors
TestHint test_hint(all_queries_text);
@ -2023,6 +2026,9 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
processTextAsSingleQuery("SET send_logs_level = 'fatal'");
}
/// Test tags are started with "--" so they are interpreted as comments anyway.
/// But if the echo is enabled we have to remove the test tags from `all_queries_text`
/// because we don't want test tags to be echoed.
size_t test_tags_length = getTestTagsLength(all_queries_text);
/// Several queries separated by ';'.

View File

@ -34,11 +34,13 @@ ConnectionPoolWithFailover::ConnectionPoolWithFailover(
{
const std::string & local_hostname = getFQDNOrHostName();
get_priority_load_balancing.hostname_differences.resize(nested_pools.size());
get_priority_load_balancing.hostname_prefix_distance.resize(nested_pools.size());
get_priority_load_balancing.hostname_levenshtein_distance.resize(nested_pools.size());
for (size_t i = 0; i < nested_pools.size(); ++i)
{
ConnectionPool & connection_pool = dynamic_cast<ConnectionPool &>(*nested_pools[i]);
get_priority_load_balancing.hostname_differences[i] = getHostNameDifference(local_hostname, connection_pool.getHost());
get_priority_load_balancing.hostname_prefix_distance[i] = getHostNamePrefixDistance(local_hostname, connection_pool.getHost());
get_priority_load_balancing.hostname_levenshtein_distance[i] = getHostNameLevenshteinDistance(local_hostname, connection_pool.getHost());
}
}

View File

@ -1,4 +1,6 @@
#include <Common/AsyncTaskExecutor.h>
#include <base/scope_guard.h>
namespace DB
{
@ -46,8 +48,10 @@ void AsyncTaskExecutor::cancel()
{
std::lock_guard guard(fiber_lock);
is_cancelled = true;
cancelBefore();
destroyFiber();
{
SCOPE_EXIT({ destroyFiber(); });
cancelBefore();
}
cancelAfter();
}

View File

@ -223,7 +223,7 @@ namespace DB
void CaresPTRResolver::process_possible_timeout(ares_channel channel)
{
/* Call ares_process() unconditonally here, even if we simply timed out
/* Call ares_process() unconditionally here, even if we simply timed out
above, as otherwise the ares name resolve won't timeout! */
ares_process_fd(channel, ARES_SOCKET_BAD, ARES_SOCKET_BAD);
}

View File

@ -971,6 +971,50 @@ Dwarf::Die Dwarf::getDieAtOffset(const CompilationUnit & cu, uint64_t offset) co
return die;
}
std::optional<std::pair<std::optional<Dwarf::CompilationUnit>, uint64_t>> Dwarf::getReferenceAttribute(
const CompilationUnit & cu, const Die & die, uint64_t attr_name) const
{
bool found = false;
uint64_t value;
uint64_t form;
forEachAttribute(cu, die, [&](const Attribute & attr)
{
if (attr.spec.name == attr_name)
{
found = true;
value = std::get<uint64_t>(attr.attr_value);
form = attr.spec.form;
return false;
}
return true;
});
if (!found)
return std::nullopt;
switch (form)
{
case DW_FORM_ref1:
case DW_FORM_ref2:
case DW_FORM_ref4:
case DW_FORM_ref8:
case DW_FORM_ref_udata:
return std::make_pair(std::nullopt, cu.offset + value);
case DW_FORM_ref_addr:
return std::make_pair(findCompilationUnit(value), value);
case DW_FORM_ref_sig8:
/// Currently we don't use this parser for types, so no need to support this.
throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Type signatures are not supported (DIE at 0x{:x}, attr 0x{:x}).", die.offset, attr_name);
case DW_FORM_ref_sup4:
case DW_FORM_ref_sup8:
throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Supplementary object files are not supported (DIE at 0x{:x}, attr 0x{:x}).", die.offset, attr_name);
default:
throw Exception(ErrorCodes::CANNOT_PARSE_DWARF, "Unexpected form of attribute 0x{:x}: 0x{:x} (DIE at 0x{:x}).", attr_name, form, die.offset);
}
}
/**
* Find the @locationInfo for @address in the compilation unit represented
* by the @sp .debug_info entry.
@ -1300,51 +1344,24 @@ void Dwarf::findInlinedSubroutineDieForAddress(
location.file = line_vm.getFullFileName(*call_file);
location.line = *call_line;
/// Something wrong with receiving debug info about inline.
/// If set to true we stop parsing DWARF.
bool die_for_inline_broken = false;
auto get_function_name = [&](const CompilationUnit & srcu, uint64_t die_offset)
{
Die decl_die = getDieAtOffset(srcu, die_offset);
auto & die_to_look_for_name = decl_die;
Die die_to_look_for_name = getDieAtOffset(srcu, die_offset);
Die def_die;
// Jump to the actual function definition instead of declaration for name
// and line info.
// DW_AT_specification: Incomplete, non-defining, or separate declaration
// corresponding to a declaration
auto offset = getAttribute<uint64_t>(srcu, decl_die, DW_AT_specification);
if (offset)
auto def = getReferenceAttribute(srcu, die_to_look_for_name, DW_AT_specification);
if (def.has_value())
{
/// FIXME: actually it's a bug in our DWARF parser.
///
/// Most of the times compilation unit offset (srcu.offset) is some big number inside .debug_info (like 434782255).
/// Offset of DIE definition is some small relative number to srcu.offset (like 3518).
/// However in some unknown cases offset looks like global, non relative number (like 434672579) and in this
/// case we obviously doing something wrong parsing DWARF.
///
/// What is important -- this bug? reproduces only with -flto=thin in release mode.
/// Also llvm-dwarfdump --verify ./clickhouse says that our DWARF is ok, so it's another prove
/// that we just doing something wrong.
///
/// FIXME: Currently we just give up parsing DWARF for inlines when we got into this situation.
if (srcu.offset + offset.value() >= info_.size())
{
die_for_inline_broken = true;
}
else
{
def_die = getDieAtOffset(srcu, srcu.offset + offset.value());
die_to_look_for_name = def_die;
}
auto [def_cu, def_offset] = std::move(def.value());
const CompilationUnit & def_cu_ref = def_cu.has_value() ? def_cu.value() : srcu;
die_to_look_for_name = getDieAtOffset(def_cu_ref, def_offset);
}
std::string_view name;
if (die_for_inline_broken)
return name;
// The file and line will be set in the next inline subroutine based on
// its DW_AT_call_file and DW_AT_call_line.
forEachAttribute(srcu, die_to_look_for_name, [&](const Attribute & attr)
@ -1386,10 +1403,6 @@ void Dwarf::findInlinedSubroutineDieForAddress(
? get_function_name(cu, cu.offset + *abstract_origin)
: get_function_name(findCompilationUnit(*abstract_origin), *abstract_origin);
/// FIXME: see comment above
if (die_for_inline_broken)
return false;
locations.push_back(location);
findInlinedSubroutineDieForAddress(cu, child_die, line_vm, address, base_addr_cu, locations, max_size);

View File

@ -453,22 +453,11 @@ private:
// Finds the Compilation Unit starting at offset.
CompilationUnit findCompilationUnit(uint64_t targetOffset) const;
template <class T>
std::optional<T> getAttribute(const CompilationUnit & cu, const Die & die, uint64_t attr_name) const
{
std::optional<T> result;
forEachAttribute(cu, die, [&](const Attribute & attr)
{
if (attr.spec.name == attr_name)
{
result = std::get<T>(attr.attr_value);
return false;
}
return true;
});
return result;
}
// Parses an attribute of "reference" form class, i.e. a reference to another DIE.
// Returns the unit containing the target DIE (nullopt if it's in the same unit as the source DIE)
// and the offset of the target DIE (relative to .debug_info, not to unit).
std::optional<std::pair<std::optional<CompilationUnit>, uint64_t>> getReferenceAttribute(
const CompilationUnit & cu, const Die & die, uint64_t attr_name) const;
// Check if the given address is in the range list at the given offset in .debug_ranges.
bool isAddrInRangeList(

View File

@ -15,9 +15,14 @@ std::function<Priority(size_t index)> GetPriorityForLoadBalancing::getPriorityFu
switch (load_balance)
{
case LoadBalancing::NEAREST_HOSTNAME:
if (hostname_differences.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "It's a bug: hostname_differences is not initialized");
get_priority = [this](size_t i) { return Priority{static_cast<Int64>(hostname_differences[i])}; };
if (hostname_prefix_distance.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "It's a bug: hostname_prefix_distance is not initialized");
get_priority = [this](size_t i) { return Priority{static_cast<Int64>(hostname_prefix_distance[i])}; };
break;
case LoadBalancing::HOSTNAME_LEVENSHTEIN_DISTANCE:
if (hostname_levenshtein_distance.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "It's a bug: hostname_levenshtein_distance is not initialized");
get_priority = [this](size_t i) { return Priority{static_cast<Int64>(hostname_levenshtein_distance[i])}; };
break;
case LoadBalancing::IN_ORDER:
get_priority = [](size_t i) { return Priority{static_cast<Int64>(i)}; };

View File

@ -13,7 +13,9 @@ public:
bool operator == (const GetPriorityForLoadBalancing & other) const
{
return load_balancing == other.load_balancing && hostname_differences == other.hostname_differences;
return load_balancing == other.load_balancing
&& hostname_prefix_distance == other.hostname_prefix_distance
&& hostname_levenshtein_distance == other.hostname_levenshtein_distance;
}
bool operator != (const GetPriorityForLoadBalancing & other) const
@ -23,7 +25,8 @@ public:
std::function<Priority(size_t index)> getPriorityFunc(LoadBalancing load_balance, size_t offset, size_t pool_size) const;
std::vector<size_t> hostname_differences; /// Distances from name of this host to the names of hosts of pools.
std::vector<size_t> hostname_prefix_distance; /// Prefix distances from name of this host to the names of hosts of pools.
std::vector<size_t> hostname_levenshtein_distance; /// Levenshtein Distances from name of this host to the names of hosts of pools.
LoadBalancing load_balancing = LoadBalancing::RANDOM;

View File

@ -997,6 +997,7 @@ protected:
--m_size;
buf[place_value].setZero();
inserted = false;
keyHolderDiscardKey(key_holder);
throw;
}
@ -1273,6 +1274,10 @@ public:
return !buf[place_value].isZero(*this);
}
bool ALWAYS_INLINE contains(const Key & x) const
{
return has(x);
}
void write(DB::WriteBuffer & wb) const
{

View File

@ -2,6 +2,7 @@
#include <base/types.h>
#include <Common/PODArray.h>
#include <Common/levenshteinDistance.h>
#include <algorithm>
#include <cctype>
@ -29,31 +30,6 @@ public:
}
private:
static size_t levenshteinDistance(const String & lhs, const String & rhs)
{
size_t m = lhs.size();
size_t n = rhs.size();
PODArrayWithStackMemory<size_t, 64> row(n + 1);
for (size_t i = 1; i <= n; ++i)
row[i] = i;
for (size_t j = 1; j <= m; ++j)
{
row[0] = j;
size_t prev = j - 1;
for (size_t i = 1; i <= n; ++i)
{
size_t old = row[i];
row[i] = std::min(prev + (std::tolower(lhs[j - 1]) != std::tolower(rhs[i - 1])),
std::min(row[i - 1], row[i]) + 1);
prev = old;
}
}
return row[n];
}
static void appendToQueue(size_t ind, const String & name, DistanceIndexQueue & queue, const std::vector<String> & prompting_strings)
{
const String & prompt = prompting_strings[ind];

View File

@ -418,7 +418,7 @@ finish:
/// this two vals are useless, xxx|xxx cannot be trivial nor prefix.
bool next_is_trivial = true;
pos = analyzeImpl(regexp, pos, required_substring, next_is_trivial, next_alternatives);
/// For xxx|xxx|xxx, we only conbine the alternatives and return a empty required_substring.
/// For xxx|xxx|xxx, we only combine the alternatives and return a empty required_substring.
if (next_alternatives.empty() || shortest_literal_length(next_alternatives) < required_substring.literal.size())
{
global_alternatives.push_back(required_substring);

View File

@ -23,7 +23,7 @@ struct ThreadEventData
UInt64 system_ms = 0;
UInt64 memory_usage = 0;
// -1 used as flag 'is not show for old servers'
// -1 used as flag 'is not shown for old servers'
Int64 peak_memory_usage = -1;
};

View File

@ -321,7 +321,7 @@ protected:
percolate(ptr);
}
// This is equivallent to one step of bubble sort
// This is equivalent to one step of bubble sort
void percolate(Counter * counter)
{
while (counter->slot > 0)

View File

@ -39,12 +39,14 @@ ZooKeeperArgs::ZooKeeperArgs(const Poco::Util::AbstractConfiguration & config, c
throw KeeperException::fromMessage(Coordination::Error::ZBADARGUMENTS, "Timeout cannot be negative");
/// init get_priority_load_balancing
get_priority_load_balancing.hostname_differences.resize(hosts.size());
get_priority_load_balancing.hostname_prefix_distance.resize(hosts.size());
get_priority_load_balancing.hostname_levenshtein_distance.resize(hosts.size());
const String & local_hostname = getFQDNOrHostName();
for (size_t i = 0; i < hosts.size(); ++i)
{
const String & node_host = hosts[i].substr(0, hosts[i].find_last_of(':'));
get_priority_load_balancing.hostname_differences[i] = DB::getHostNameDifference(local_hostname, node_host);
get_priority_load_balancing.hostname_prefix_distance[i] = DB::getHostNamePrefixDistance(local_hostname, node_host);
get_priority_load_balancing.hostname_levenshtein_distance[i] = DB::getHostNameLevenshteinDistance(local_hostname, node_host);
}
}

View File

@ -5,6 +5,7 @@
#include <optional>
#include <base/types.h>
#include <Common/Exception.h>
#include <Common/levenshteinDistance.h>
#include <Poco/Net/IPAddress.h>
#include <Poco/Net/SocketAddress.h>
@ -121,10 +122,8 @@ bool isLocalAddress(const Poco::Net::SocketAddress & address, UInt16 clickhouse_
return clickhouse_port == address.port() && isLocalAddress(address.host());
}
size_t getHostNameDifference(const std::string & local_hostname, const std::string & host)
size_t getHostNamePrefixDistance(const std::string & local_hostname, const std::string & host)
{
/// FIXME should we replace it with Levenstein distance? (we already have it in NamePrompter)
size_t hostname_difference = 0;
for (size_t i = 0; i < std::min(local_hostname.length(), host.length()); ++i)
if (local_hostname[i] != host[i])
@ -132,4 +131,9 @@ size_t getHostNameDifference(const std::string & local_hostname, const std::stri
return hostname_difference;
}
size_t getHostNameLevenshteinDistance(const std::string & local_hostname, const std::string & host)
{
return levenshteinDistance(local_hostname, host);
}
}

View File

@ -26,6 +26,8 @@ namespace DB
bool isLocalAddress(const Poco::Net::SocketAddress & address);
bool isLocalAddress(const Poco::Net::IPAddress & address);
/// Returns number of different bytes in hostnames, used for load balancing
size_t getHostNameDifference(const std::string & local_hostname, const std::string & host);
/// Returns host name difference with name prefix, used for load balancing
size_t getHostNamePrefixDistance(const std::string & local_hostname, const std::string & host);
/// Returns host name difference with Levenshtein Distance.
size_t getHostNameLevenshteinDistance(const std::string & local_hostname, const std::string & host);
}

View File

@ -0,0 +1,32 @@
#include <Common/levenshteinDistance.h>
#include <Common/PODArray.h>
namespace DB
{
size_t levenshteinDistance(const String & lhs, const String & rhs)
{
size_t m = lhs.size();
size_t n = rhs.size();
PODArrayWithStackMemory<size_t, 64> row(n + 1);
for (size_t i = 1; i <= n; ++i)
row[i] = i;
for (size_t j = 1; j <= m; ++j)
{
row[0] = j;
size_t prev = j - 1;
for (size_t i = 1; i <= n; ++i)
{
size_t old = row[i];
row[i] = std::min(prev + (std::tolower(lhs[j - 1]) != std::tolower(rhs[i - 1])),
std::min(row[i - 1], row[i]) + 1);
prev = old;
}
}
return row[n];
}
}

View File

@ -0,0 +1,12 @@
#pragma once
#include <base/types.h>
namespace DB
{
/// How many steps if we want to change lhs to rhs.
/// Details in https://en.wikipedia.org/wiki/Levenshtein_distance
size_t levenshteinDistance(const String & lhs, const String & rhs);
}

View File

@ -9,7 +9,6 @@ TEST(EventNotifier, SimpleTest)
using namespace DB;
size_t result = 1;
EventNotifier::init();
auto handler3 = EventNotifier::instance().subscribe(Coordination::Error::ZSESSIONEXPIRED, [&result](){ result *= 3; });

View File

@ -0,0 +1,18 @@
#include <gtest/gtest.h>
#include <Common/tests/gtest_global_context.h>
class ContextEnvironment : public testing::Environment
{
public:
void SetUp() override { getContext(); }
};
int main(int argc, char ** argv)
{
testing::InitGoogleTest(&argc, argv);
testing::AddGlobalTestEnvironment(new ContextEnvironment);
return RUN_ALL_TESTS();
}

View File

@ -3,6 +3,7 @@
#include <Common/Config/ConfigProcessor.h>
#include <Common/Macros.h>
#include <Common/ThreadPool.h>
#include <Common/callOnce.h>
#include <Core/ServerSettings.h>
@ -14,6 +15,7 @@
namespace ProfileEvents
{
extern const Event ContextLock;
extern const Event ContextLockWaitMicroseconds;
}
namespace CurrentMetrics
@ -39,8 +41,8 @@ struct ContextSharedPart : boost::noncopyable
: macros(std::make_unique<Macros>())
{}
/// For access of most of shared objects. Recursive mutex.
mutable std::recursive_mutex mutex;
/// For access of most of shared objects.
mutable SharedMutex mutex;
mutable std::mutex keeper_dispatcher_mutex;
mutable std::shared_ptr<KeeperDispatcher> keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex);
@ -50,13 +52,16 @@ struct ContextSharedPart : boost::noncopyable
String path; /// Path to the data directory, with a slash at the end.
ConfigurationPtr config; /// Global configuration settings.
MultiVersion<Macros> macros; /// Substitutions extracted from config.
OnceFlag schedule_pool_initialized;
mutable std::unique_ptr<BackgroundSchedulePool> schedule_pool; /// A thread pool that can run different jobs in background
RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml
///
mutable OnceFlag readers_initialized;
mutable std::unique_ptr<IAsynchronousReader> asynchronous_remote_fs_reader;
mutable std::unique_ptr<IAsynchronousReader> asynchronous_local_fs_reader;
mutable std::unique_ptr<IAsynchronousReader> synchronous_local_fs_reader;
mutable OnceFlag threadpool_writer_initialized;
mutable std::unique_ptr<ThreadPool> threadpool_writer;
mutable ThrottlerPtr remote_read_throttler; /// A server-wide throttler for remote IO reads
@ -64,13 +69,14 @@ struct ContextSharedPart : boost::noncopyable
mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads
mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes
};
ContextData::ContextData() = default;
ContextData::ContextData(const ContextData &) = default;
Context::Context() = default;
Context::Context(const Context & rhs) : ContextData(rhs), std::enable_shared_from_this<Context>(rhs) {}
Context::~Context() = default;
Context::Context(const Context &) = default;
Context & Context::operator=(const Context &) = default;
SharedContextHolder::SharedContextHolder(SharedContextHolder &&) noexcept = default;
SharedContextHolder & SharedContextHolder::operator=(SharedContextHolder &&) noexcept = default;
@ -87,10 +93,10 @@ void Context::makeGlobalContext()
global_context = shared_from_this();
}
ContextMutablePtr Context::createGlobal(ContextSharedPart * shared)
ContextMutablePtr Context::createGlobal(ContextSharedPart * shared_part)
{
auto res = std::shared_ptr<Context>(new Context);
res->shared = shared;
res->shared = shared_part;
return res;
}
@ -105,6 +111,7 @@ SharedContextHolder Context::createShared()
return SharedContextHolder(std::make_unique<ContextSharedPart>());
}
ContextMutablePtr Context::getGlobalContext() const
{
auto ptr = global_context.lock();
@ -112,22 +119,55 @@ ContextMutablePtr Context::getGlobalContext() const
return ptr;
}
std::unique_lock<std::recursive_mutex> Context::getLock() const
std::unique_lock<SharedMutex> Context::getGlobalLock() const
{
ProfileEvents::increment(ProfileEvents::ContextLock);
CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait};
return std::unique_lock(shared->mutex);
Stopwatch watch;
auto lock = std::unique_lock(shared->mutex);
ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds());
return lock;
}
std::shared_lock<SharedMutex> Context::getGlobalSharedLock() const
{
ProfileEvents::increment(ProfileEvents::ContextLock);
CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait};
Stopwatch watch;
auto lock = std::shared_lock(shared->mutex);
ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds());
return lock;
}
std::unique_lock<SharedMutex> Context::getLocalLock() const
{
ProfileEvents::increment(ProfileEvents::ContextLock);
CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait};
Stopwatch watch;
auto lock = std::unique_lock(mutex);
ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds());
return lock;
}
std::shared_lock<SharedMutex> Context::getLocalSharedLock() const
{
ProfileEvents::increment(ProfileEvents::ContextLock);
CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait};
Stopwatch watch;
auto lock = std::shared_lock(mutex);
ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds());
return lock;
}
String Context::getPath() const
{
auto lock = getLock();
auto lock = getGlobalSharedLock();
return shared->path;
}
void Context::setPath(const String & path)
{
auto lock = getLock();
auto lock = getGlobalLock();
shared->path = path;
}
@ -143,15 +183,13 @@ void Context::setMacros(std::unique_ptr<Macros> && macros)
BackgroundSchedulePool & Context::getSchedulePool() const
{
auto lock = getLock();
if (!shared->schedule_pool)
{
callOnce(shared->schedule_pool_initialized, [&] {
shared->schedule_pool = std::make_unique<BackgroundSchedulePool>(
shared->server_settings.background_schedule_pool_size,
CurrentMetrics::BackgroundSchedulePoolTask,
CurrentMetrics::BackgroundSchedulePoolSize,
"BgSchPool");
}
});
return *shared->schedule_pool;
}
@ -168,30 +206,21 @@ const RemoteHostFilter & Context::getRemoteHostFilter() const
IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) const
{
auto lock = getLock();
callOnce(shared->readers_initialized, [&] {
const auto & config = getConfigRef();
shared->asynchronous_remote_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER, config);
shared->asynchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER, config);
shared->synchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER, config);
});
switch (type)
{
case FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER:
{
if (!shared->asynchronous_remote_fs_reader)
shared->asynchronous_remote_fs_reader = createThreadPoolReader(type, getConfigRef());
return *shared->asynchronous_remote_fs_reader;
}
case FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER:
{
if (!shared->asynchronous_local_fs_reader)
shared->asynchronous_local_fs_reader = createThreadPoolReader(type, getConfigRef());
return *shared->asynchronous_local_fs_reader;
}
case FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER:
{
if (!shared->synchronous_local_fs_reader)
shared->synchronous_local_fs_reader = createThreadPoolReader(type, getConfigRef());
return *shared->synchronous_local_fs_reader;
}
}
}
@ -207,19 +236,19 @@ std::shared_ptr<FilesystemReadPrefetchesLog> Context::getFilesystemReadPrefetche
void Context::setConfig(const ConfigurationPtr & config)
{
auto lock = getLock();
auto lock = getGlobalLock();
shared->config = config;
}
const Poco::Util::AbstractConfiguration & Context::getConfigRef() const
{
auto lock = getLock();
auto lock = getGlobalSharedLock();
return shared->config ? *shared->config : Poco::Util::Application::instance().config();
}
std::shared_ptr<AsyncReadCounters> Context::getAsyncReadCounters() const
{
auto lock = getLock();
auto lock = getLocalLock();
if (!async_read_counters)
async_read_counters = std::make_shared<AsyncReadCounters>();
return async_read_counters;
@ -227,18 +256,14 @@ std::shared_ptr<AsyncReadCounters> Context::getAsyncReadCounters() const
ThreadPool & Context::getThreadPoolWriter() const
{
const auto & config = getConfigRef();
auto lock = getLock();
if (!shared->threadpool_writer)
{
callOnce(shared->threadpool_writer_initialized, [&] {
const auto & config = getConfigRef();
auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100);
auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000);
shared->threadpool_writer = std::make_unique<ThreadPool>(
CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, pool_size, pool_size, queue_size);
}
});
return *shared->threadpool_writer;
}

View File

@ -6,6 +6,7 @@
#include <Common/MultiVersion.h>
#include <Common/RemoteHostFilter.h>
#include <Common/SharedMutex.h>
#include <Disks/IO/getThreadPoolReader.h>
@ -44,17 +45,9 @@ private:
std::unique_ptr<ContextSharedPart> shared;
};
class Context : public std::enable_shared_from_this<Context>
class ContextData
{
private:
/// Use copy constructor or createGlobal() instead
Context();
Context(const Context &);
Context & operator=(const Context &);
std::unique_lock<std::recursive_mutex> getLock() const;
protected:
ContextWeakMutablePtr global_context;
inline static ContextPtr global_context_instance;
ContextSharedPart * shared;
@ -63,9 +56,33 @@ private:
mutable std::shared_ptr<AsyncReadCounters> async_read_counters;
Settings settings; /// Setting for query execution.
public:
/// Use copy constructor or createGlobal() instead
ContextData();
ContextData(const ContextData &);
};
class Context : public ContextData, public std::enable_shared_from_this<Context>
{
private:
/// ContextData mutex
mutable SharedMutex mutex;
Context();
Context(const Context &);
std::unique_lock<SharedMutex> getGlobalLock() const;
std::shared_lock<SharedMutex> getGlobalSharedLock() const;
std::unique_lock<SharedMutex> getLocalLock() const;
std::shared_lock<SharedMutex> getLocalSharedLock() const;
public:
/// Create initial Context with ContextShared and etc.
static ContextMutablePtr createGlobal(ContextSharedPart * shared);
static ContextMutablePtr createGlobal(ContextSharedPart * shared_part);
static SharedContextHolder createShared();
ContextMutablePtr getGlobalContext() const;

View File

@ -284,7 +284,7 @@ void deserializeLogMagic(ReadBuffer & in)
/// strange, that this 550 bytes obviously was a part of Create transaction,
/// but the operation code was -1. We have added debug prints to original
/// zookeeper (3.6.3) and found that it just reads 550 bytes of this "Error"
/// transaction, tooks the first 4 bytes as an error code (it was 79, non
/// transaction, took the first 4 bytes as an error code (it was 79, non
/// existing code) and skip all remaining 546 bytes. NOTE: it looks like a bug
/// in ZooKeeper.
///

View File

@ -71,6 +71,13 @@ protected:
DB::KeeperContextPtr keeper_context = std::make_shared<DB::KeeperContext>(true);
Poco::Logger * log{&Poco::Logger::get("CoordinationTest")};
void SetUp() override
{
Poco::AutoPtr<Poco::ConsoleChannel> channel(new Poco::ConsoleChannel(std::cerr));
Poco::Logger::root().setChannel(channel);
Poco::Logger::root().setLevel("trace");
}
void setLogDirectory(const std::string & path) { keeper_context->setLogDisk(std::make_shared<DB::DiskLocal>("LogDisk", path)); }
void setSnapshotDirectory(const std::string & path)
@ -2911,13 +2918,4 @@ INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite,
CoordinationTest,
::testing::ValuesIn(std::initializer_list<CompressionParam>{CompressionParam{true, ".zstd"}, CompressionParam{false, ""}}));
int main(int argc, char ** argv)
{
Poco::AutoPtr<Poco::ConsoleChannel> channel(new Poco::ConsoleChannel(std::cerr));
Poco::Logger::root().setChannel(channel);
Poco::Logger::root().setLevel("trace");
testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}
#endif

View File

@ -0,0 +1,65 @@
#include "Common/assert_cast.h"
#include "Columns/ColumnNullable.h"
#include "Columns/ColumnsDateTime.h"
#include "Core/DecimalFunctions.h"
#include "DataTypes/IDataType.h"
#include "base/types.h"
namespace DB
{
namespace MySQLProtocol
{
namespace MySQLUtils
{
DecimalUtils::DecimalComponents<DateTime64>
getNormalizedDateTime64Components(DataTypePtr data_type, ColumnPtr col, size_t row_num)
{
const auto * date_time_type = typeid_cast<const DataTypeDateTime64 *>(data_type.get());
static constexpr UInt32 MaxScale = DecimalUtils::max_precision<DateTime64>;
UInt32 scale = std::min(MaxScale, date_time_type->getScale());
const auto value = assert_cast<const ColumnDateTime64 &>(*col).getData()[row_num];
auto components = DecimalUtils::split(value, scale);
using T = typename DateTime64::NativeType;
if (value.value < 0 && components.fractional)
{
components.fractional = DecimalUtils::scaleMultiplier<T>(scale) + (components.whole ? T(-1) : T(1)) * components.fractional;
--components.whole;
}
if (components.fractional != 0)
{
if (scale > 6)
{
// MySQL Timestamp has max scale of 6
components.fractional /= static_cast<int>(pow(10, scale - 6));
}
else
{
// fractional == 1 is a different microsecond value depending on the scale
// Scale 1 = 100000
// Scale 2 = 010000
// Scale 3 = 001000
// Scale 4 = 000100
// Scale 5 = 000010
// Scale 6 = 000001
components.fractional *= static_cast<int>(pow(10, 6 - scale));
}
}
return components;
};
ColumnPtr getBaseColumn(const DB::Columns & columns, size_t i)
{
ColumnPtr col = columns[i]->convertToFullIfNeeded();
if (col->isNullable())
return assert_cast<const ColumnNullable &>(*col).getNestedColumnPtr();
return col;
};
}
}
}

View File

@ -0,0 +1,22 @@
#pragma once
#include "Core/DecimalFunctions.h"
#include "DataTypes/IDataType.h"
#include "base/types.h"
namespace DB
{
namespace MySQLProtocol
{
namespace MySQLUtils
{
/// Splits DateTime64 column data at a certain row number into whole and fractional part
/// Additionally, normalizes the fractional part as if it was scale 6 for MySQL compatibility purposes
DecimalUtils::DecimalComponents<DateTime64> getNormalizedDateTime64Components(DataTypePtr data_type, ColumnPtr col, size_t row_num);
/// If a column is ColumnSparse/ColumnLowCardinality/ColumnNullable, it is unwrapped in a correct order;
/// otherwise, the original column is returned
ColumnPtr getBaseColumn(const DB::Columns & columns, size_t i);
}
}
}

View File

@ -15,6 +15,7 @@
#include "DataTypes/DataTypesNumber.h"
#include "Formats/FormatSettings.h"
#include "IO/WriteBufferFromString.h"
#include "MySQLUtils.h"
#include "base/DayNum.h"
#include "base/Decimal.h"
#include "base/types.h"
@ -25,14 +26,14 @@ namespace MySQLProtocol
{
namespace ProtocolBinary
{
ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTypes & data_types_, const Columns & columns_, int row_num_)
ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTypes & data_types_, const Columns & columns_, size_t row_num_)
: row_num(row_num_), columns(columns_), data_types(data_types_), serializations(serializations_)
{
payload_size = 1 + null_bitmap_size;
FormatSettings format_settings;
for (size_t i = 0; i < columns.size(); ++i)
{
ColumnPtr col = getColumn(i);
ColumnPtr col = MySQLUtils::getBaseColumn(columns, i);
if (col->isNullAt(row_num))
{
// See https://dev.mysql.com/doc/dev/mysql-server/8.1.0/page_protocol_binary_resultset.html#sect_protocol_binary_resultset_row
@ -42,7 +43,7 @@ ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTyp
continue; // NULLs are stored in the null bitmap only
}
DataTypePtr data_type = removeLowCardinality(removeNullable((data_types[i])));
DataTypePtr data_type = removeLowCardinalityAndNullable(data_types[i]);
TypeIndex type_index = data_type->getTypeId();
switch (type_index)
{
@ -50,10 +51,7 @@ ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTyp
payload_size += 1;
break;
case TypeIndex::UInt8:
if (data_type->getName() == "Bool")
payload_size += 2; // BIT MySQL type is string<lenenc> in binary
else
payload_size += 1;
payload_size += 1;
break;
case TypeIndex::Int16:
case TypeIndex::UInt16:
@ -98,13 +96,7 @@ ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTyp
break;
}
case TypeIndex::DateTime64: {
auto [components, scale] = getDateTime64ComponentsWithScale(data_type, col);
if (scale > 6)
{
// MySQL Timestamp has max scale of 6
components.fractional /= static_cast<int>(pow(10, scale - 6));
}
auto components = MySQLUtils::getNormalizedDateTime64Components(data_type, col, row_num);
LocalDateTime ldt = LocalDateTime(components.whole, DateLUT::instance(getDateTimeTimezone(*data_type)));
bool has_microseconds = components.fractional != 0;
@ -153,11 +145,11 @@ void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const
buffer.write(null_bitmap.data(), null_bitmap_size);
for (size_t i = 0; i < columns.size(); ++i)
{
ColumnPtr col = getColumn(i);
ColumnPtr col = MySQLUtils::getBaseColumn(columns, i);
if (col->isNullAt(row_num))
continue;
DataTypePtr data_type = removeLowCardinality(removeNullable((data_types[i])));
DataTypePtr data_type = removeLowCardinalityAndNullable(data_types[i]);
TypeIndex type_index = data_type->getTypeId();
switch (type_index)
{
@ -168,8 +160,6 @@ void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const
}
case TypeIndex::UInt8: {
UInt8 value = assert_cast<const ColumnVector<UInt8> &>(*col).getData()[row_num];
if (data_type->getName() == "Bool")
buffer.write(static_cast<char>(1));
buffer.write(reinterpret_cast<char *>(&value), 1);
break;
}
@ -262,27 +252,7 @@ void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const
break;
}
case TypeIndex::DateTime64: {
auto [components, scale] = getDateTime64ComponentsWithScale(data_type, col);
if (components.fractional != 0)
{
if (scale > 6)
{
// MySQL Timestamp has max scale of 6
components.fractional /= static_cast<int>(pow(10, scale - 6));
}
else
{
// fractional == 1 is a different microsecond value depending on the scale
// Scale 1 = 100000
// Scale 2 = 010000
// Scale 3 = 001000
// Scale 4 = 000100
// Scale 5 = 000010
// Scale 6 = 000001
components.fractional *= static_cast<int>(pow(10, 6 - scale));
}
}
auto components = MySQLUtils::getNormalizedDateTime64Components(data_type, col, row_num);
String timezone = getDateTimeTimezone(*data_type);
LocalDateTime ldt = LocalDateTime(components.whole, DateLUT::instance(timezone));
UInt16 year = ldt.year();
@ -332,34 +302,6 @@ void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const
}
}
}
ResultSetRow::DateTime64ComponentsWithScale ResultSetRow::getDateTime64ComponentsWithScale(DataTypePtr data_type, ColumnPtr col) const
{
const auto * date_time_type = typeid_cast<const DataTypeDateTime64 *>(data_type.get());
static constexpr UInt32 MaxScale = DecimalUtils::max_precision<DateTime64>;
UInt32 scale = std::min(MaxScale, date_time_type->getScale());
const auto value = assert_cast<const ColumnDateTime64 &>(*col).getData()[row_num];
auto components = DecimalUtils::split(value, scale);
using T = typename DateTime64::NativeType;
if (value.value < 0 && components.fractional)
{
components.fractional = DecimalUtils::scaleMultiplier<T>(scale) + (components.whole ? T(-1) : T(1)) * components.fractional;
--components.whole;
}
return {components, scale};
}
ColumnPtr ResultSetRow::getColumn(size_t i) const
{
ColumnPtr col = columns[i]->convertToFullIfNeeded();
if (col->isNullable())
return assert_cast<const ColumnNullable &>(*col).getNestedColumnPtr();
return col;
}
}
}
}

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