mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge branch 'master' into pr-enable-local-plan
This commit is contained in:
commit
3d4c663c56
2
contrib/orc
vendored
2
contrib/orc
vendored
@ -1 +1 @@
|
||||
Subproject commit bcc025c09828c556f54cfbdf83a66b9acae7d17f
|
||||
Subproject commit 223e1957ff308f2aec7793884ffb0d7692d48484
|
@ -56,7 +56,7 @@ Parameters:
|
||||
distance](https://en.wikipedia.org/wiki/Cosine_similarity#Cosine_distance)- the angle between two non-zero vectors).
|
||||
- `quantization`: either `f64`, `f32`, `f16`, `bf16`, or `i8` for storing the vector with reduced precision (optional, default: `bf16`)
|
||||
- `hnsw_max_connections_per_layer`: the number of neighbors per HNSW graph node, also known as `M` in the [HNSW
|
||||
paper](https://doi.org/10.1109/TPAMI.2018.2889473) (optional, default: 16)
|
||||
paper](https://doi.org/10.1109/TPAMI.2018.2889473) (optional, default: 32)
|
||||
- `hnsw_candidate_list_size_for_construction`: the size of the dynamic candidate list when constructing the HNSW graph, also known as
|
||||
`ef_construction` in the original [HNSW paper](https://doi.org/10.1109/TPAMI.2018.2889473) (optional, default: 128)
|
||||
|
||||
@ -143,7 +143,7 @@ clickhouse-client --param_vec='hello' --query="SELECT * FROM table WHERE L2Dista
|
||||
```
|
||||
:::
|
||||
|
||||
To search using a different value of HNSW parameter `hnsw_candidate_list_size_for_search` (default: 64), also known as `ef_search` in the
|
||||
To search using a different value of HNSW parameter `hnsw_candidate_list_size_for_search` (default: 256), also known as `ef_search` in the
|
||||
original [HNSW paper](https://doi.org/10.1109/TPAMI.2018.2889473), run the `SELECT` query with `SETTINGS hnsw_candidate_list_size_for_search
|
||||
= <value>`.
|
||||
|
||||
|
@ -10,12 +10,15 @@ A popular benchmark which models the internal data warehouse of a wholesale supp
|
||||
The data is stored into a 3rd normal form representation, requiring lots of joins at query runtime.
|
||||
Despite its age and its unrealistic assumption that the data is uniformly and independently distributed, TPC-H remains the most popular OLAP benchmark to date.
|
||||
|
||||
References
|
||||
**References**
|
||||
|
||||
- [TPC-H](https://www.tpc.org/tpc_documents_current_versions/current_specifications5.asp)
|
||||
- [New TPC Benchmarks for Decision Support and Web Commerce](https://doi.org/10.1145/369275.369291) (Poess et. al., 2000)
|
||||
- [TPC-H Analyzed: Hidden Messages and Lessons Learned from an Influential Benchmark](https://doi.org/10.1007/978-3-319-04936-6_5) (Boncz et. al.), 2013
|
||||
- [Quantifying TPC-H Choke Points and Their Optimizations](https://doi.org/10.14778/3389133.3389138) (Dresseler et. al.), 2020
|
||||
|
||||
## Data Generation and Import
|
||||
|
||||
First, checkout the TPC-H repository and compile the data generator:
|
||||
|
||||
``` bash
|
||||
@ -30,7 +33,17 @@ Then, generate the data. Parameter `-s` specifies the scale factor. For example,
|
||||
./dbgen -s 100
|
||||
```
|
||||
|
||||
Now create tables in ClickHouse:
|
||||
Now create tables in ClickHouse.
|
||||
|
||||
We stick as closely as possible to the rules of the TPC-H specification:
|
||||
- Primary keys are created only for the columns mentioned in section 1.4.2.2 of the specification.
|
||||
- Substitution parameters were replaced by the values for query validation in sections 2.1.x.4 of the specification.
|
||||
- As per section 1.4.2.1, the table definitions do not use the optional `NOT NULL` constraints, even if `dbgen` generates them by default.
|
||||
The performance of `SELECT` queries in ClickHouse is not affected by the presence or absence of `NOT NULL` constraints.
|
||||
- As per section 1.3.1, we use ClickHouse's native datatypes (e.g. `Int32`, `String`) to implement the abstract datatypes mentioned in the
|
||||
specification (e.g. `Identifier`, `Variable text, size N`). The only effect of this is better readability, the SQL-92 datatypes generated
|
||||
by `dbgen` (e.g. `INTEGER`, `VARCHAR(40)`) would also work in ClickHouse.
|
||||
|
||||
|
||||
```sql
|
||||
CREATE TABLE nation (
|
||||
@ -38,13 +51,13 @@ CREATE TABLE nation (
|
||||
n_name String,
|
||||
n_regionkey Int32,
|
||||
n_comment String)
|
||||
ORDER BY (n_regionkey, n_name);
|
||||
ORDER BY (n_nationkey);
|
||||
|
||||
CREATE TABLE region (
|
||||
r_regionkey Int32,
|
||||
r_name String,
|
||||
r_comment String)
|
||||
ORDER BY (r_name);
|
||||
ORDER BY (r_regionkey);
|
||||
|
||||
CREATE TABLE part (
|
||||
p_partkey Int32,
|
||||
@ -56,7 +69,7 @@ CREATE TABLE part (
|
||||
p_container String,
|
||||
p_retailprice Decimal(15,2),
|
||||
p_comment String)
|
||||
ORDER BY (p_mfgr, p_brand, p_type, p_name);
|
||||
ORDER BY (p_partkey);
|
||||
|
||||
CREATE TABLE supplier (
|
||||
s_suppkey Int32,
|
||||
@ -66,7 +79,7 @@ CREATE TABLE supplier (
|
||||
s_phone String,
|
||||
s_acctbal Decimal(15,2),
|
||||
s_comment String)
|
||||
ORDER BY (s_nationkey, s_address, s_name);
|
||||
ORDER BY (s_suppkey);
|
||||
|
||||
CREATE TABLE partsupp (
|
||||
ps_partkey Int32,
|
||||
@ -74,7 +87,7 @@ CREATE TABLE partsupp (
|
||||
ps_availqty Int32,
|
||||
ps_supplycost Decimal(15,2),
|
||||
ps_comment String)
|
||||
ORDER BY (ps_suppkey, ps_availqty, ps_supplycost, ps_partkey);
|
||||
ORDER BY (ps_partkey, ps_suppkey);
|
||||
|
||||
CREATE TABLE customer (
|
||||
c_custkey Int32,
|
||||
@ -85,7 +98,7 @@ CREATE TABLE customer (
|
||||
c_acctbal Decimal(15,2),
|
||||
c_mktsegment String,
|
||||
c_comment String)
|
||||
ORDER BY (c_nationkey, c_mktsegment, c_address, c_name, c_custkey);
|
||||
ORDER BY (c_custkey);
|
||||
|
||||
CREATE TABLE orders (
|
||||
o_orderkey Int32,
|
||||
@ -97,7 +110,10 @@ CREATE TABLE orders (
|
||||
o_clerk String,
|
||||
o_shippriority Int32,
|
||||
o_comment String)
|
||||
ORDER BY (o_orderdate, o_orderstatus, o_custkey);
|
||||
ORDER BY (o_orderkey);
|
||||
-- The following is an alternative order key which is not compliant with the official TPC-H rules but recommended by sec. 4.5 in
|
||||
-- "Quantifying TPC-H Choke Points and Their Optimizations":
|
||||
-- ORDER BY (o_orderdate, o_orderkey);
|
||||
|
||||
CREATE TABLE lineitem (
|
||||
l_orderkey Int32,
|
||||
@ -116,7 +132,10 @@ CREATE TABLE lineitem (
|
||||
l_shipinstruct String,
|
||||
l_shipmode String,
|
||||
l_comment String)
|
||||
ORDER BY (l_suppkey, l_partkey, l_shipdate, l_commitdate, l_receiptdate);
|
||||
ORDER BY (l_orderkey, l_linenumber);
|
||||
-- The following is an alternative order key which is not compliant with the official TPC-H rules but recommended by sec. 4.5 in
|
||||
-- "Quantifying TPC-H Choke Points and Their Optimizations":
|
||||
-- ORDER BY (l_shipdate, l_orderkey, l_linenumber);
|
||||
```
|
||||
|
||||
The data can be imported as follows:
|
||||
@ -127,19 +146,21 @@ clickhouse-client --format_csv_delimiter '|' --query "INSERT INTO region FORMAT
|
||||
clickhouse-client --format_csv_delimiter '|' --query "INSERT INTO part FORMAT CSV" < part.tbl
|
||||
clickhouse-client --format_csv_delimiter '|' --query "INSERT INTO supplier FORMAT CSV" < supplier.tbl
|
||||
clickhouse-client --format_csv_delimiter '|' --query "INSERT INTO partsupp FORMAT CSV" < partsupp.tbl
|
||||
clickhouse-client --format_csv_delimiter '|' --query "INSERT INTO customers FORMAT CSV" < customers.tbl
|
||||
clickhouse-client --format_csv_delimiter '|' --query "INSERT INTO customer FORMAT CSV" < customer.tbl
|
||||
clickhouse-client --format_csv_delimiter '|' --query "INSERT INTO orders FORMAT CSV" < orders.tbl
|
||||
clickhouse-client --format_csv_delimiter '|' --query "INSERT INTO lineitem FORMAT CSV" < lineitem.tbl
|
||||
```
|
||||
|
||||
The queries are generated by `./qgen -s <scaling_factor>`. Example queries for `s = 100`:
|
||||
|
||||
## Queries
|
||||
|
||||
::::warning
|
||||
TPC-H makes heavy use of correlated subqueries which are at the time of writing (September 2024) not supported by ClickHouse ([issue #6697](https://github.com/ClickHouse/ClickHouse/issues/6697)).
|
||||
TPC-H makes heavy use of correlated subqueries which are at the time of writing (October 2024) not supported by ClickHouse ([issue #6697](https://github.com/ClickHouse/ClickHouse/issues/6697)).
|
||||
As a result, many of below benchmark queries will fail with errors.
|
||||
::::
|
||||
|
||||
Q1
|
||||
**Q1**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -156,7 +177,7 @@ SELECT
|
||||
FROM
|
||||
lineitem
|
||||
WHERE
|
||||
l_shipdate <= date '1998-12-01' - interval '100' day
|
||||
l_shipdate <= date '1998-12-01' - INTERVAL '90' DAY
|
||||
GROUP BY
|
||||
l_returnflag,
|
||||
l_linestatus
|
||||
@ -165,7 +186,7 @@ ORDER BY
|
||||
l_linestatus;
|
||||
```
|
||||
|
||||
Q2
|
||||
**Q2**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -186,11 +207,11 @@ FROM
|
||||
WHERE
|
||||
p_partkey = ps_partkey
|
||||
AND s_suppkey = ps_suppkey
|
||||
AND p_size = 21
|
||||
AND p_type LIKE '%COPPER'
|
||||
AND p_size = 15
|
||||
AND p_type LIKE '%BRASS'
|
||||
AND s_nationkey = n_nationkey
|
||||
AND n_regionkey = r_regionkey
|
||||
AND r_name = 'AMERICA'
|
||||
AND r_name = 'EUROPE'
|
||||
AND ps_supplycost = (
|
||||
SELECT
|
||||
min(ps_supplycost)
|
||||
@ -204,17 +225,16 @@ WHERE
|
||||
AND s_suppkey = ps_suppkey
|
||||
AND s_nationkey = n_nationkey
|
||||
AND n_regionkey = r_regionkey
|
||||
AND r_name = 'AMERICA'
|
||||
AND r_name = 'EUROPE'
|
||||
)
|
||||
ORDER BY
|
||||
s_acctbal desc,
|
||||
s_acctbal DESC,
|
||||
n_name,
|
||||
s_name,
|
||||
p_partkey
|
||||
LIMIT 100;
|
||||
p_partkey;
|
||||
```
|
||||
|
||||
Q3
|
||||
**Q3**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -230,19 +250,18 @@ WHERE
|
||||
c_mktsegment = 'BUILDING'
|
||||
AND c_custkey = o_custkey
|
||||
AND l_orderkey = o_orderkey
|
||||
AND o_orderdate < date '1995-03-10'
|
||||
AND l_shipdate > date '1995-03-10'
|
||||
AND o_orderdate < date '1995-03-15'
|
||||
AND l_shipdate > date '1995-03-15'
|
||||
GROUP BY
|
||||
l_orderkey,
|
||||
o_orderdate,
|
||||
o_shippriority
|
||||
ORDER BY
|
||||
revenue desc,
|
||||
o_orderdate
|
||||
LIMIT 10;
|
||||
revenue DESC,
|
||||
o_orderdate;
|
||||
```
|
||||
|
||||
Q4
|
||||
**Q4**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -251,8 +270,8 @@ SELECT
|
||||
FROM
|
||||
orders
|
||||
WHERE
|
||||
o_orderdate >= date '1994-07-01'
|
||||
AND o_orderdate < date '1994-07-01' + interval '3' month
|
||||
o_orderdate >= date '1993-07-01'
|
||||
AND o_orderdate < date '1993-07-01' + INTERVAL '3' MONTH
|
||||
AND EXISTS (
|
||||
SELECT
|
||||
*
|
||||
@ -268,7 +287,7 @@ ORDER BY
|
||||
o_orderpriority;
|
||||
```
|
||||
|
||||
Q5
|
||||
**Q5**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -288,16 +307,16 @@ WHERE
|
||||
AND c_nationkey = s_nationkey
|
||||
AND s_nationkey = n_nationkey
|
||||
AND n_regionkey = r_regionkey
|
||||
AND r_name = 'MIDDLE EAST'
|
||||
AND r_name = 'ASIA'
|
||||
AND o_orderdate >= date '1994-01-01'
|
||||
AND o_orderdate < date '1994-01-01' + interval '1' year
|
||||
AND o_orderdate < date '1994-01-01' + INTERVAL '1' year
|
||||
GROUP BY
|
||||
n_name
|
||||
ORDER BY
|
||||
revenue desc;
|
||||
revenue DESC;
|
||||
```
|
||||
|
||||
Q6
|
||||
**Q6**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -306,12 +325,12 @@ FROM
|
||||
lineitem
|
||||
WHERE
|
||||
l_shipdate >= date '1994-01-01'
|
||||
AND l_shipdate < date '1994-01-01' + interval '1' year
|
||||
AND l_discount between 0.09 - 0.01 AND 0.09 + 0.01
|
||||
AND l_shipdate < date '1994-01-01' + INTERVAL '1' year
|
||||
AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01
|
||||
AND l_quantity < 24;
|
||||
```
|
||||
|
||||
Q7
|
||||
**Q7**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -319,31 +338,30 @@ SELECT
|
||||
cust_nation,
|
||||
l_year,
|
||||
sum(volume) AS revenue
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
n1.n_name AS supp_nation,
|
||||
n2.n_name AS cust_nation,
|
||||
extract(year FROM l_shipdate) AS l_year,
|
||||
l_extendedprice * (1 - l_discount) AS volume
|
||||
FROM
|
||||
supplier,
|
||||
lineitem,
|
||||
orders,
|
||||
customer,
|
||||
nation n1,
|
||||
nation n2
|
||||
WHERE
|
||||
s_suppkey = l_suppkey
|
||||
AND o_orderkey = l_orderkey
|
||||
AND c_custkey = o_custkey
|
||||
AND s_nationkey = n1.n_nationkey
|
||||
AND c_nationkey = n2.n_nationkey
|
||||
AND (
|
||||
(n1.n_name = 'UNITED KINGDOM' AND n2.n_name = 'ETHIOPIA')
|
||||
OR (n1.n_name = 'ETHIOPIA' AND n2.n_name = 'UNITED KINGDOM')
|
||||
)
|
||||
AND l_shipdate between date '1995-01-01' AND date '1996-12-31'
|
||||
FROM (
|
||||
SELECT
|
||||
n1.n_name AS supp_nation,
|
||||
n2.n_name AS cust_nation,
|
||||
extract(year FROM l_shipdate) AS l_year,
|
||||
l_extendedprice * (1 - l_discount) AS volume
|
||||
FROM
|
||||
supplier,
|
||||
lineitem,
|
||||
orders,
|
||||
customer,
|
||||
nation n1,
|
||||
nation n2
|
||||
WHERE
|
||||
s_suppkey = l_suppkey
|
||||
AND o_orderkey = l_orderkey
|
||||
AND c_custkey = o_custkey
|
||||
AND s_nationkey = n1.n_nationkey
|
||||
AND c_nationkey = n2.n_nationkey
|
||||
AND (
|
||||
(n1.n_name = 'FRANCE' AND n2.n_name = 'GERMANY')
|
||||
OR (n1.n_name = 'GERMANY' AND n2.n_name = 'FRANCE')
|
||||
)
|
||||
AND l_shipdate BETWEEN date '1995-01-01' AND date '1996-12-31'
|
||||
) AS shipping
|
||||
GROUP BY
|
||||
supp_nation,
|
||||
@ -355,41 +373,41 @@ ORDER BY
|
||||
l_year;
|
||||
```
|
||||
|
||||
Q8
|
||||
**Q8**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
o_year,
|
||||
sum(CASE
|
||||
WHEN nation = 'ETHIOPIA' THEN volume
|
||||
ELSE 0
|
||||
END) / sum(volume) AS mkt_share
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
extract(year FROM o_orderdate) AS o_year,
|
||||
l_extendedprice * (1 - l_discount) AS volume,
|
||||
n2.n_name AS nation
|
||||
FROM
|
||||
part,
|
||||
supplier,
|
||||
lineitem,
|
||||
orders,
|
||||
customer,
|
||||
nation n1,
|
||||
nation n2,
|
||||
region
|
||||
WHERE
|
||||
p_partkey = l_partkey
|
||||
AND s_suppkey = l_suppkey
|
||||
AND l_orderkey = o_orderkey
|
||||
AND o_custkey = c_custkey
|
||||
AND c_nationkey = n1.n_nationkey
|
||||
AND n1.n_regionkey = r_regionkey
|
||||
AND r_name = 'AFRICA'
|
||||
AND s_nationkey = n2.n_nationkey
|
||||
AND o_orderdate between date '1995-01-01' AND date '1996-12-31'
|
||||
AND p_type = 'SMALL POLISHED TIN'
|
||||
WHEN nation = 'BRAZIL'
|
||||
THEN volume
|
||||
ELSE 0
|
||||
END) / sum(volume) AS mkt_share
|
||||
FROM (
|
||||
SELECT
|
||||
extract(year FROM o_orderdate) AS o_year,
|
||||
l_extendedprice * (1 - l_discount) AS volume,
|
||||
n2.n_name AS nation
|
||||
FROM
|
||||
part,
|
||||
supplier,
|
||||
lineitem,
|
||||
orders,
|
||||
customer,
|
||||
nation n1,
|
||||
nation n2,
|
||||
region
|
||||
WHERE
|
||||
p_partkey = l_partkey
|
||||
AND s_suppkey = l_suppkey
|
||||
AND l_orderkey = o_orderkey
|
||||
AND o_custkey = c_custkey
|
||||
AND c_nationkey = n1.n_nationkey
|
||||
AND n1.n_regionkey = r_regionkey
|
||||
AND r_name = 'AMERICA'
|
||||
AND s_nationkey = n2.n_nationkey
|
||||
AND o_orderdate BETWEEN date '1995-01-01' AND date '1996-12-31'
|
||||
AND p_type = 'ECONOMY ANODIZED STEEL'
|
||||
) AS all_nations
|
||||
GROUP BY
|
||||
o_year
|
||||
@ -397,44 +415,43 @@ ORDER BY
|
||||
o_year;
|
||||
```
|
||||
|
||||
Q9
|
||||
**Q9**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
nation,
|
||||
o_year,
|
||||
sum(amount) AS sum_profit
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
n_name AS nation,
|
||||
extract(year FROM o_orderdate) AS o_year,
|
||||
l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity AS amount
|
||||
FROM
|
||||
part,
|
||||
supplier,
|
||||
lineitem,
|
||||
partsupp,
|
||||
orders,
|
||||
nation
|
||||
WHERE
|
||||
s_suppkey = l_suppkey
|
||||
AND ps_suppkey = l_suppkey
|
||||
AND ps_partkey = l_partkey
|
||||
AND p_partkey = l_partkey
|
||||
AND o_orderkey = l_orderkey
|
||||
AND s_nationkey = n_nationkey
|
||||
AND p_name LIKE '%drab%'
|
||||
FROM (
|
||||
SELECT
|
||||
n_name AS nation,
|
||||
extract(year FROM o_orderdate) AS o_year,
|
||||
l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity AS amount
|
||||
FROM
|
||||
part,
|
||||
supplier,
|
||||
lineitem,
|
||||
partsupp,
|
||||
orders,
|
||||
nation
|
||||
WHERE
|
||||
s_suppkey = l_suppkey
|
||||
AND ps_suppkey = l_suppkey
|
||||
AND ps_partkey = l_partkey
|
||||
AND p_partkey = l_partkey
|
||||
AND o_orderkey = l_orderkey
|
||||
AND s_nationkey = n_nationkey
|
||||
AND p_name LIKE '%green%'
|
||||
) AS profit
|
||||
GROUP BY
|
||||
nation,
|
||||
o_year
|
||||
ORDER BY
|
||||
nation,
|
||||
o_year desc;
|
||||
o_year DESC;
|
||||
```
|
||||
|
||||
Q10
|
||||
**Q10**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -454,8 +471,8 @@ FROM
|
||||
WHERE
|
||||
c_custkey = o_custkey
|
||||
AND l_orderkey = o_orderkey
|
||||
AND o_orderdate >= date '1993-06-01'
|
||||
AND o_orderdate < date '1993-06-01' + interval '3' month
|
||||
AND o_orderdate >= date '1993-10-01'
|
||||
AND o_orderdate < date '1993-10-01' + INTERVAL '3' MONTH
|
||||
AND l_returnflag = 'R'
|
||||
AND c_nationkey = n_nationkey
|
||||
GROUP BY
|
||||
@ -467,11 +484,10 @@ GROUP BY
|
||||
c_address,
|
||||
c_comment
|
||||
ORDER BY
|
||||
revenue desc
|
||||
LIMIT 20;
|
||||
revenue DESC;
|
||||
```
|
||||
|
||||
Q11
|
||||
**Q11**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -484,12 +500,12 @@ FROM
|
||||
WHERE
|
||||
ps_suppkey = s_suppkey
|
||||
AND s_nationkey = n_nationkey
|
||||
AND n_name = 'MOZAMBIQUE'
|
||||
AND n_name = 'GERMANY'
|
||||
GROUP BY
|
||||
ps_partkey having
|
||||
ps_partkey HAVING
|
||||
sum(ps_supplycost * ps_availqty) > (
|
||||
SELECT
|
||||
sum(ps_supplycost * ps_availqty) * 0.0000010000
|
||||
sum(ps_supplycost * ps_availqty) * 0.0001
|
||||
FROM
|
||||
partsupp,
|
||||
supplier,
|
||||
@ -497,89 +513,88 @@ GROUP BY
|
||||
WHERE
|
||||
ps_suppkey = s_suppkey
|
||||
AND s_nationkey = n_nationkey
|
||||
AND n_name = 'MOZAMBIQUE'
|
||||
AND n_name = 'GERMANY'
|
||||
)
|
||||
ORDER BY
|
||||
value desc;
|
||||
value DESC;
|
||||
```
|
||||
|
||||
Q12
|
||||
**Q12**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
l_shipmode,
|
||||
sum(CASE
|
||||
WHEN o_orderpriority = '1-URGENT'
|
||||
OR o_orderpriority = '2-HIGH'
|
||||
WHEN o_orderpriority = '1-URGENT'
|
||||
OR o_orderpriority = '2-HIGH'
|
||||
THEN 1
|
||||
ELSE 0
|
||||
END) AS high_line_count,
|
||||
ELSE 0
|
||||
END) AS high_line_count,
|
||||
sum(CASE
|
||||
WHEN o_orderpriority <> '1-URGENT'
|
||||
AND o_orderpriority <> '2-HIGH'
|
||||
AND o_orderpriority <> '2-HIGH'
|
||||
THEN 1
|
||||
ELSE 0
|
||||
END) AS low_line_count
|
||||
END) AS low_line_count
|
||||
FROM
|
||||
orders,
|
||||
lineitem
|
||||
WHERE
|
||||
o_orderkey = l_orderkey
|
||||
AND l_shipmode in ('MAIL', 'AIR')
|
||||
AND l_shipmode in ('MAIL', 'SHIP')
|
||||
AND l_commitdate < l_receiptdate
|
||||
AND l_shipdate < l_commitdate
|
||||
AND l_receiptdate >= date '1996-01-01'
|
||||
AND l_receiptdate < date '1996-01-01' + interval '1' year
|
||||
AND l_receiptdate >= date '1994-01-01'
|
||||
AND l_receiptdate < date '1994-01-01' + INTERVAL '1' year
|
||||
GROUP BY
|
||||
l_shipmode
|
||||
ORDER BY
|
||||
l_shipmode;
|
||||
```
|
||||
|
||||
Q13
|
||||
**Q13**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
c_count,
|
||||
count(*) AS custdist
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
c_custkey,
|
||||
count(o_orderkey)
|
||||
FROM
|
||||
customer LEFT OUTER JOIN orders ON
|
||||
c_custkey = o_custkey
|
||||
AND o_comment NOT LIKE '%special%deposits%'
|
||||
GROUP BY
|
||||
c_custkey
|
||||
FROM (
|
||||
SELECT
|
||||
c_custkey,
|
||||
count(o_orderkey)
|
||||
FROM
|
||||
customer LEFT OUTER JOIN orders ON
|
||||
c_custkey = o_custkey
|
||||
AND o_comment NOT LIKE '%special%requests%'
|
||||
GROUP BY
|
||||
c_custkey
|
||||
) AS c_orders
|
||||
GROUP BY
|
||||
c_count
|
||||
ORDER BY
|
||||
custdist desc,
|
||||
c_count desc;
|
||||
custdist DESC,
|
||||
c_count DESC;
|
||||
```
|
||||
|
||||
Q14
|
||||
**Q14**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
100.00 * sum(CASE
|
||||
WHEN p_type LIKE 'PROMO%'
|
||||
THEN l_extendedprice * (1 - l_discount)
|
||||
ELSE 0
|
||||
END) / sum(l_extendedprice * (1 - l_discount)) AS promo_revenue
|
||||
WHEN p_type LIKE 'PROMO%'
|
||||
THEN l_extendedprice * (1 - l_discount)
|
||||
ELSE 0
|
||||
END) / sum(l_extendedprice * (1 - l_discount)) AS promo_revenue
|
||||
FROM
|
||||
lineitem,
|
||||
part
|
||||
WHERE
|
||||
l_partkey = p_partkey
|
||||
AND l_shipdate >= date '1996-10-01'
|
||||
AND l_shipdate < date '1996-10-01' + interval '1' month;
|
||||
AND l_shipdate >= date '1995-09-01'
|
||||
AND l_shipdate < date '1995-09-01' + INTERVAL '1' MONTH;
|
||||
```
|
||||
|
||||
Q15
|
||||
**Q15**
|
||||
|
||||
```sql
|
||||
CREATE VIEW revenue0 (supplier_no, total_revenue) AS
|
||||
@ -589,8 +604,8 @@ CREATE VIEW revenue0 (supplier_no, total_revenue) AS
|
||||
FROM
|
||||
lineitem
|
||||
WHERE
|
||||
l_shipdate >= date '1997-06-01'
|
||||
AND l_shipdate < date '1997-06-01' + interval '3' month
|
||||
l_shipdate >= date '1996-01-01'
|
||||
AND l_shipdate < date '1996-01-01' + INTERVAL '3' MONTH
|
||||
GROUP BY
|
||||
l_suppkey;
|
||||
|
||||
@ -617,7 +632,7 @@ ORDER BY
|
||||
DROP VIEW revenue0;
|
||||
```
|
||||
|
||||
Q16
|
||||
**Q16**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -630,9 +645,9 @@ FROM
|
||||
part
|
||||
WHERE
|
||||
p_partkey = ps_partkey
|
||||
AND p_brand <> 'Brand#15'
|
||||
AND p_type NOT LIKE 'SMALL POLISHED%'
|
||||
AND p_size in (21, 9, 46, 34, 50, 33, 17, 36)
|
||||
AND p_brand <> 'Brand#45'
|
||||
AND p_type NOT LIKE 'MEDIUM POLISHED%'
|
||||
AND p_size in (49, 14, 23, 45, 19, 3, 36, 9)
|
||||
AND ps_suppkey NOT in (
|
||||
SELECT
|
||||
s_suppkey
|
||||
@ -646,13 +661,13 @@ GROUP BY
|
||||
p_type,
|
||||
p_size
|
||||
ORDER BY
|
||||
supplier_cnt desc,
|
||||
supplier_cnt DESC,
|
||||
p_brand,
|
||||
p_type,
|
||||
p_size;
|
||||
```
|
||||
|
||||
Q17
|
||||
**Q17**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -662,8 +677,8 @@ FROM
|
||||
part
|
||||
WHERE
|
||||
p_partkey = l_partkey
|
||||
AND p_brand = 'Brand#52'
|
||||
AND p_container = 'MED CASE'
|
||||
AND p_brand = 'Brand#23'
|
||||
AND p_container = 'MED BOX'
|
||||
AND l_quantity < (
|
||||
SELECT
|
||||
0.2 * avg(l_quantity)
|
||||
@ -674,7 +689,7 @@ WHERE
|
||||
);
|
||||
```
|
||||
|
||||
Q18
|
||||
**Q18**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -695,8 +710,9 @@ WHERE
|
||||
FROM
|
||||
lineitem
|
||||
GROUP BY
|
||||
l_orderkey having
|
||||
sum(l_quantity) > 313
|
||||
l_orderkey
|
||||
HAVING
|
||||
sum(l_quantity) > 300
|
||||
)
|
||||
AND c_custkey = o_custkey
|
||||
AND o_orderkey = l_orderkey
|
||||
@ -707,12 +723,11 @@ GROUP BY
|
||||
o_orderdate,
|
||||
o_totalprice
|
||||
ORDER BY
|
||||
o_totalprice desc,
|
||||
o_orderdate
|
||||
LIMIT 100;
|
||||
o_totalprice DESC,
|
||||
o_orderdate;
|
||||
```
|
||||
|
||||
Q19
|
||||
**Q19**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -723,36 +738,36 @@ FROM
|
||||
WHERE
|
||||
(
|
||||
p_partkey = l_partkey
|
||||
AND p_brand = 'Brand#31'
|
||||
AND p_brand = 'Brand#12'
|
||||
AND p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')
|
||||
AND l_quantity >= 3 AND l_quantity <= 3 + 10
|
||||
AND p_size between 1 AND 5
|
||||
AND l_quantity >= 1 AND l_quantity <= 1 + 10
|
||||
AND p_size BETWEEN 1 AND 5
|
||||
AND l_shipmode in ('AIR', 'AIR REG')
|
||||
AND l_shipinstruct = 'DELIVER IN PERSON'
|
||||
)
|
||||
OR
|
||||
(
|
||||
p_partkey = l_partkey
|
||||
AND p_brand = 'Brand#54'
|
||||
AND p_brand = 'Brand#23'
|
||||
AND p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')
|
||||
AND l_quantity >= 17 AND l_quantity <= 17 + 10
|
||||
AND p_size between 1 AND 10
|
||||
AND l_quantity >= 10 AND l_quantity <= 10 + 10
|
||||
AND p_size BETWEEN 1 AND 10
|
||||
AND l_shipmode in ('AIR', 'AIR REG')
|
||||
AND l_shipinstruct = 'DELIVER IN PERSON'
|
||||
)
|
||||
OR
|
||||
(
|
||||
p_partkey = l_partkey
|
||||
AND p_brand = 'Brand#54'
|
||||
AND p_brand = 'Brand#34'
|
||||
AND p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')
|
||||
AND l_quantity >= 26 AND l_quantity <= 26 + 10
|
||||
AND p_size between 1 AND 15
|
||||
AND l_quantity >= 20 AND l_quantity <= 20 + 10
|
||||
AND p_size BETWEEN 1 AND 15
|
||||
AND l_shipmode in ('AIR', 'AIR REG')
|
||||
AND l_shipinstruct = 'DELIVER IN PERSON'
|
||||
);
|
||||
```
|
||||
|
||||
Q20
|
||||
**Q20**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -774,7 +789,7 @@ WHERE
|
||||
FROM
|
||||
part
|
||||
WHERE
|
||||
p_name LIKE 'chiffon%'
|
||||
p_name LIKE 'forrest%'
|
||||
)
|
||||
AND ps_availqty > (
|
||||
SELECT
|
||||
@ -784,17 +799,17 @@ WHERE
|
||||
WHERE
|
||||
l_partkey = ps_partkey
|
||||
AND l_suppkey = ps_suppkey
|
||||
AND l_shipdate >= date '1997-01-01'
|
||||
AND l_shipdate < date '1997-01-01' + interval '1' year
|
||||
AND l_shipdate >= date '1994-01-01'
|
||||
AND l_shipdate < date '1994-01-01' + INTERVAL '1' year
|
||||
)
|
||||
)
|
||||
AND s_nationkey = n_nationkey
|
||||
AND n_name = 'MOZAMBIQUE'
|
||||
AND n_name = 'CANADA'
|
||||
ORDER BY
|
||||
s_name;
|
||||
```
|
||||
|
||||
Q21
|
||||
**Q21**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -830,16 +845,15 @@ WHERE
|
||||
AND l3.l_receiptdate > l3.l_commitdate
|
||||
)
|
||||
AND s_nationkey = n_nationkey
|
||||
AND n_name = 'RUSSIA'
|
||||
AND n_name = 'SAUDI ARABIA'
|
||||
GROUP BY
|
||||
s_name
|
||||
ORDER BY
|
||||
numwait desc,
|
||||
s_name
|
||||
LIMIT 100;
|
||||
numwait DESC,
|
||||
s_name;
|
||||
```
|
||||
|
||||
Q22
|
||||
**Q22**
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
@ -855,7 +869,7 @@ FROM
|
||||
customer
|
||||
WHERE
|
||||
substring(c_phone FROM 1 for 2) in
|
||||
('26', '34', '10', '18', '27', '12', '11')
|
||||
('13', '31', '23', '29', '30', '18', '17')
|
||||
AND c_acctbal > (
|
||||
SELECT
|
||||
avg(c_acctbal)
|
||||
@ -864,7 +878,7 @@ FROM
|
||||
WHERE
|
||||
c_acctbal > 0.00
|
||||
AND substring(c_phone FROM 1 for 2) in
|
||||
('26', '34', '10', '18', '27', '12', '11')
|
||||
('13', '31', '23', '29', '30', '18', '17')
|
||||
)
|
||||
AND NOT EXISTS (
|
||||
SELECT
|
||||
|
@ -18,7 +18,7 @@ Parameters:
|
||||
|
||||
Returned values:
|
||||
|
||||
Constants `(a, b)` of the resulting line `y = a*x + b`.
|
||||
Constants `(k, b)` of the resulting line `y = k*x + b`.
|
||||
|
||||
**Examples**
|
||||
|
||||
|
@ -4393,7 +4393,7 @@ Result:
|
||||
|
||||
## globalVariable
|
||||
|
||||
Takes constant string argument and returns the value of global variable with that name. It is intended for compatibility with MySQL.
|
||||
Takes a constant string argument and returns the value of the global variable with that name. This function is intended for compatibility with MySQL and not needed or useful for normal operation of ClickHouse. Only few dummy global variables are defined.
|
||||
|
||||
**Syntax**
|
||||
|
||||
|
@ -18,7 +18,7 @@ simpleLinearRegression(x, y)
|
||||
|
||||
Возвращаемые значения:
|
||||
|
||||
Константы `(a, b)` результирующей прямой `y = a*x + b`.
|
||||
Константы `(k, b)` результирующей прямой `y = k*x + b`.
|
||||
|
||||
**Примеры**
|
||||
|
||||
|
@ -20,7 +20,7 @@ simpleLinearRegression(x, y)
|
||||
|
||||
**返回值**
|
||||
|
||||
符合`y = a*x + b`的常量 `(a, b)` 。
|
||||
符合`y = k*x + b`的常量 `(k, b)` 。
|
||||
|
||||
**示例**
|
||||
|
||||
|
@ -184,6 +184,26 @@ public:
|
||||
, join_node(join_node_)
|
||||
{}
|
||||
|
||||
bool needChildVisit(const QueryTreeNodePtr & parent, const QueryTreeNodePtr &)
|
||||
{
|
||||
/** Optimization can change the value of some expression from NULL to FALSE.
|
||||
* For example:
|
||||
* when `a` is `NULL`, the expression `a = b AND a IS NOT NULL` returns `NULL`
|
||||
* and it will be optimized to `a = b`, which returns `FALSE`.
|
||||
* This is valid for JOIN ON condition and for the functions `AND`/`OR` inside it.
|
||||
* (When we replace `AND`/`OR` operands from `NULL` to `FALSE`, the result value can also change only from `NULL` to `FALSE`)
|
||||
* However, in the general case, the result can be wrong.
|
||||
* For example, for NOT: `NOT NULL` is `NULL`, but `NOT FALSE` is `TRUE`.
|
||||
* Therefore, optimize only top-level expression or expressions inside `AND`/`OR`.
|
||||
*/
|
||||
if (const auto * function_node = parent->as<FunctionNode>())
|
||||
{
|
||||
const auto & func_name = function_node->getFunctionName();
|
||||
return func_name == "or" || func_name == "and";
|
||||
}
|
||||
return parent->getNodeType() == QueryTreeNodeType::LIST;
|
||||
}
|
||||
|
||||
void enterImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
auto * function_node = node->as<FunctionNode>();
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <IO/WriteBufferFromS3.h>
|
||||
#include <IO/HTTPHeaderEntries.h>
|
||||
#include <IO/S3/copyS3File.h>
|
||||
#include <IO/S3/deleteFileFromS3.h>
|
||||
#include <IO/S3/Client.h>
|
||||
#include <IO/S3/Credentials.h>
|
||||
#include <Disks/IDisk.h>
|
||||
@ -117,12 +118,6 @@ namespace
|
||||
throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType());
|
||||
return outcome.GetResult().GetContents();
|
||||
}
|
||||
|
||||
bool isNotFoundError(Aws::S3::S3Errors error)
|
||||
{
|
||||
return error == Aws::S3::S3Errors::RESOURCE_NOT_FOUND
|
||||
|| error == Aws::S3::S3Errors::NO_SUCH_KEY;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -236,6 +231,7 @@ BackupWriterS3::BackupWriterS3(
|
||||
: BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterS3"))
|
||||
, s3_uri(s3_uri_)
|
||||
, data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false}
|
||||
, s3_capabilities(getCapabilitiesFromConfig(context_->getConfigRef(), "s3"))
|
||||
{
|
||||
s3_settings.loadFromConfig(context_->getConfigRef(), "s3", context_->getSettingsRef());
|
||||
|
||||
@ -358,92 +354,22 @@ std::unique_ptr<WriteBuffer> BackupWriterS3::writeFile(const String & file_name)
|
||||
|
||||
void BackupWriterS3::removeFile(const String & file_name)
|
||||
{
|
||||
S3::DeleteObjectRequest request;
|
||||
request.SetBucket(s3_uri.bucket);
|
||||
auto key = fs::path(s3_uri.key) / file_name;
|
||||
request.SetKey(key);
|
||||
|
||||
auto outcome = client->DeleteObject(request);
|
||||
|
||||
if (blob_storage_log)
|
||||
{
|
||||
blob_storage_log->addEvent(
|
||||
BlobStorageLogElement::EventType::Delete,
|
||||
s3_uri.bucket, key, /* local_path */ "", /* data_size */ 0,
|
||||
outcome.IsSuccess() ? nullptr : &outcome.GetError());
|
||||
}
|
||||
|
||||
if (!outcome.IsSuccess() && !isNotFoundError(outcome.GetError().GetErrorType()))
|
||||
throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType());
|
||||
deleteFileFromS3(client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, /* if_exists = */ false,
|
||||
blob_storage_log);
|
||||
}
|
||||
|
||||
void BackupWriterS3::removeFiles(const Strings & file_names)
|
||||
{
|
||||
try
|
||||
{
|
||||
if (!supports_batch_delete.has_value() || supports_batch_delete.value() == true)
|
||||
{
|
||||
removeFilesBatch(file_names);
|
||||
supports_batch_delete = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const auto & file_name : file_names)
|
||||
removeFile(file_name);
|
||||
}
|
||||
}
|
||||
catch (const Exception &)
|
||||
{
|
||||
if (!supports_batch_delete.has_value())
|
||||
{
|
||||
supports_batch_delete = false;
|
||||
LOG_TRACE(log, "DeleteObjects is not supported. Retrying with plain DeleteObject.");
|
||||
Strings keys;
|
||||
keys.reserve(file_names.size());
|
||||
for (const String & file_name : file_names)
|
||||
keys.push_back(fs::path(s3_uri.key) / file_name);
|
||||
|
||||
for (const auto & file_name : file_names)
|
||||
removeFile(file_name);
|
||||
}
|
||||
else
|
||||
throw;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void BackupWriterS3::removeFilesBatch(const Strings & file_names)
|
||||
{
|
||||
/// One call of DeleteObjects() cannot remove more than 1000 keys.
|
||||
size_t chunk_size_limit = 1000;
|
||||
size_t batch_size = 1000;
|
||||
|
||||
size_t current_position = 0;
|
||||
while (current_position < file_names.size())
|
||||
{
|
||||
std::vector<Aws::S3::Model::ObjectIdentifier> current_chunk;
|
||||
for (; current_position < file_names.size() && current_chunk.size() < chunk_size_limit; ++current_position)
|
||||
{
|
||||
Aws::S3::Model::ObjectIdentifier obj;
|
||||
obj.SetKey(fs::path(s3_uri.key) / file_names[current_position]);
|
||||
current_chunk.push_back(obj);
|
||||
}
|
||||
|
||||
Aws::S3::Model::Delete delkeys;
|
||||
delkeys.SetObjects(current_chunk);
|
||||
S3::DeleteObjectsRequest request;
|
||||
request.SetBucket(s3_uri.bucket);
|
||||
request.SetDelete(delkeys);
|
||||
|
||||
auto outcome = client->DeleteObjects(request);
|
||||
|
||||
if (blob_storage_log)
|
||||
{
|
||||
const auto * outcome_error = outcome.IsSuccess() ? nullptr : &outcome.GetError();
|
||||
auto time_now = std::chrono::system_clock::now();
|
||||
for (const auto & obj : current_chunk)
|
||||
blob_storage_log->addEvent(BlobStorageLogElement::EventType::Delete, s3_uri.bucket, obj.GetKey(),
|
||||
/* local_path */ "", /* data_size */ 0, outcome_error, time_now);
|
||||
}
|
||||
|
||||
if (!outcome.IsSuccess() && !isNotFoundError(outcome.GetError().GetErrorType()))
|
||||
throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType());
|
||||
}
|
||||
deleteFilesFromS3(client, s3_uri.bucket, keys, /* if_exists = */ false,
|
||||
s3_capabilities, batch_size, blob_storage_log);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <IO/S3Settings.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <IO/S3/BlobStorageLogWriter.h>
|
||||
#include <IO/S3/S3Capabilities.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -76,14 +77,12 @@ public:
|
||||
|
||||
private:
|
||||
std::unique_ptr<ReadBuffer> readFile(const String & file_name, size_t expected_file_size) override;
|
||||
void removeFilesBatch(const Strings & file_names);
|
||||
|
||||
const S3::URI s3_uri;
|
||||
const DataSourceDescription data_source_description;
|
||||
S3Settings s3_settings;
|
||||
std::shared_ptr<S3::Client> client;
|
||||
std::optional<bool> supports_batch_delete;
|
||||
|
||||
S3Capabilities s3_capabilities;
|
||||
BlobStorageLogWriterPtr blob_storage_log;
|
||||
};
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteIntText.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperConstants.h>
|
||||
@ -64,14 +65,13 @@ namespace ErrorCodes
|
||||
M(UInt64, log_slow_connection_operation_threshold_ms, 1000, "Log message if a certain operation took too long inside a single connection", 0)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS)
|
||||
|
||||
struct CoordinationSettingsImpl : public BaseSettings<CoordinationSettingsTraits>
|
||||
{
|
||||
void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config);
|
||||
};
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS)
|
||||
|
||||
void CoordinationSettingsImpl::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
if (!config.has(config_elem))
|
||||
@ -99,7 +99,7 @@ void CoordinationSettingsImpl::loadFromConfig(const String & config_elem, const
|
||||
}
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
|
||||
CoordinationSettings##TYPE NAME = &CoordinationSettings##Impl ::NAME;
|
||||
CoordinationSettings##TYPE NAME = &CoordinationSettingsImpl ::NAME;
|
||||
|
||||
namespace CoordinationSetting
|
||||
{
|
||||
@ -119,18 +119,7 @@ CoordinationSettings::CoordinationSettings(const CoordinationSettings & settings
|
||||
|
||||
CoordinationSettings::~CoordinationSettings() = default;
|
||||
|
||||
#define IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR(CLASS_NAME, TYPE) \
|
||||
const SettingField##TYPE & CoordinationSettings::operator[](CLASS_NAME##TYPE t) const \
|
||||
{ \
|
||||
return impl.get()->*t; \
|
||||
} \
|
||||
SettingField##TYPE & CoordinationSettings::operator[](CLASS_NAME##TYPE t) \
|
||||
{ \
|
||||
return impl.get()->*t; \
|
||||
}
|
||||
|
||||
COORDINATION_SETTINGS_SUPPORTED_TYPES(CoordinationSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
#undef IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR
|
||||
|
||||
void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
|
@ -25,19 +25,51 @@ class WriteBuffer;
|
||||
* Example of usage:
|
||||
*
|
||||
* mysettings.h:
|
||||
* #include <Core/BaseSettingsFwdMacros.h>
|
||||
* #include <Core/SettingsFields.h>
|
||||
*
|
||||
* #define MY_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
* M(CLASS_NAME, Float) \
|
||||
* M(CLASS_NAME, String) \
|
||||
* M(CLASS_NAME, UInt64)
|
||||
*
|
||||
* MY_SETTINGS_SUPPORTED_TYPES(MySettings, DECLARE_SETTING_TRAIT)
|
||||
*
|
||||
* struct MySettings
|
||||
* {
|
||||
* MySettings();
|
||||
* ~MySettings();
|
||||
*
|
||||
* MY_SETTINGS_SUPPORTED_TYPES(MySettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
* private:
|
||||
* std::unique_ptr<MySettingsImpl> impl;
|
||||
* };
|
||||
*
|
||||
* mysettings.cpp:
|
||||
* #include <Core/BaseSettings.h>
|
||||
* #include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
*
|
||||
* #define APPLY_FOR_MYSETTINGS(M) \
|
||||
* M(UInt64, a, 100, "Description of a", 0) \
|
||||
* M(Float, f, 3.11, "Description of f", IMPORTANT) // IMPORTANT - means the setting can't be ignored by older versions) \
|
||||
* M(String, s, "default", "Description of s", 0)
|
||||
*
|
||||
* DECLARE_SETTINGS_TRAITS(MySettingsTraits, APPLY_FOR_MYSETTINGS)
|
||||
|
||||
* struct MySettings : public BaseSettings<MySettingsTraits>
|
||||
* IMPLEMENT_SETTINGS_TRAITS(MySettingsTraits, APPLY_FOR_MYSETTINGS)
|
||||
*
|
||||
* struct MySettingsImpl : public BaseSettings<MySettingsTraits>
|
||||
* {
|
||||
* };
|
||||
*
|
||||
* mysettings.cpp:
|
||||
* IMPLEMENT_SETTINGS_TRAITS(MySettingsTraits, APPLY_FOR_MYSETTINGS)
|
||||
* #define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) MySettings##TYPE NAME = &MySettings##Impl ::NAME;
|
||||
*
|
||||
* namespace MySetting
|
||||
* {
|
||||
* APPLY_FOR_MYSETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
* }
|
||||
* #undef INITIALIZE_SETTING_EXTERN
|
||||
*
|
||||
* MY_SETTINGS_SUPPORTED_TYPES(MySettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
*/
|
||||
template <class TTraits>
|
||||
class BaseSettings : public TTraits::Data
|
||||
|
11
src/Core/BaseSettingsFwdMacrosImpl.h
Normal file
11
src/Core/BaseSettingsFwdMacrosImpl.h
Normal file
@ -0,0 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#define IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR(CLASS_NAME, TYPE) \
|
||||
const SettingField##TYPE & CLASS_NAME::operator[](CLASS_NAME##TYPE t) const \
|
||||
{ \
|
||||
return impl.get()->*t; \
|
||||
} \
|
||||
SettingField##TYPE & CLASS_NAME::operator[](CLASS_NAME##TYPE t) \
|
||||
{ \
|
||||
return impl.get()->*t; \
|
||||
}
|
@ -8,11 +8,9 @@ namespace DB
|
||||
* User-specified file format settings for File and URL engines.
|
||||
*/
|
||||
DECLARE_SETTINGS_TRAITS(FormatFactorySettingsTraits, LIST_OF_ALL_FORMAT_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(FormatFactorySettingsTraits, LIST_OF_ALL_FORMAT_SETTINGS)
|
||||
|
||||
struct FormatFactorySettingsImpl : public BaseSettings<FormatFactorySettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(FormatFactorySettingsTraits, LIST_OF_ALL_FORMAT_SETTINGS)
|
||||
|
||||
}
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <IO/MMappedFileCache.h>
|
||||
#include <IO/UncompressedCache.h>
|
||||
@ -193,14 +194,13 @@ namespace DB
|
||||
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in dumpToSystemServerSettingsColumns below
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, LIST_OF_SERVER_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(ServerSettingsTraits, LIST_OF_SERVER_SETTINGS)
|
||||
|
||||
struct ServerSettingsImpl : public BaseSettings<ServerSettingsTraits>
|
||||
{
|
||||
void loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config);
|
||||
};
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(ServerSettingsTraits, LIST_OF_SERVER_SETTINGS)
|
||||
|
||||
void ServerSettingsImpl::loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
// settings which can be loaded from the the default profile, see also MAKE_DEPRECATED_BY_SERVER_CONFIG in src/Core/Settings.h
|
||||
@ -231,7 +231,7 @@ void ServerSettingsImpl::loadSettingsFromConfig(const Poco::Util::AbstractConfig
|
||||
}
|
||||
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) ServerSettings##TYPE NAME = &ServerSettings##Impl ::NAME;
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) ServerSettings##TYPE NAME = &ServerSettingsImpl ::NAME;
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
@ -250,18 +250,7 @@ ServerSettings::ServerSettings(const ServerSettings & settings) : impl(std::make
|
||||
|
||||
ServerSettings::~ServerSettings() = default;
|
||||
|
||||
#define IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR(CLASS_NAME, TYPE) \
|
||||
const SettingField##TYPE & ServerSettings::operator[](CLASS_NAME##TYPE t) const \
|
||||
{ \
|
||||
return impl.get()->*t; \
|
||||
} \
|
||||
SettingField##TYPE & ServerSettings::operator[](CLASS_NAME##TYPE t) \
|
||||
{ \
|
||||
return impl.get()->*t; \
|
||||
}
|
||||
|
||||
SERVER_SETTINGS_SUPPORTED_TYPES(ServerSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
#undef IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR
|
||||
|
||||
void ServerSettings::set(std::string_view name, const Field & value)
|
||||
{
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Core/BaseSettingsProgramOptions.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/Settings.h>
|
||||
@ -5559,8 +5560,8 @@ Only in ClickHouse Cloud. Allow to create ShareSet and SharedJoin
|
||||
M(UInt64, max_limit_for_ann_queries, 1'000'000, R"(
|
||||
SELECT queries with LIMIT bigger than this setting cannot use vector similarity indexes. Helps to prevent memory overflows in vector similarity indexes.
|
||||
)", 0) \
|
||||
M(UInt64, hnsw_candidate_list_size_for_search, 0, R"(
|
||||
The size of the dynamic candidate list when searching the vector similarity index, also known as 'ef_search'. 0 means USearch's default value (64).
|
||||
M(UInt64, hnsw_candidate_list_size_for_search, 256, R"(
|
||||
The size of the dynamic candidate list when searching the vector similarity index, also known as 'ef_search'.
|
||||
)", 0) \
|
||||
M(Bool, throw_on_unsupported_query_inside_transaction, true, R"(
|
||||
Throw exception if unsupported query is used inside transaction
|
||||
@ -5899,7 +5900,7 @@ Allow writing simple SELECT queries without the leading SELECT keyword, which ma
|
||||
OBSOLETE_FORMAT_SETTINGS(M, ALIAS) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS_ALLOW_CUSTOM_SETTINGS(SettingsTraits, LIST_OF_SETTINGS)
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(SettingsTraits, LIST_OF_SETTINGS)
|
||||
|
||||
/** Settings of query execution.
|
||||
* These settings go to users.xml.
|
||||
@ -5933,9 +5934,6 @@ private:
|
||||
std::unordered_set<std::string_view> settings_changed_by_compatibility_setting;
|
||||
};
|
||||
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(SettingsTraits, LIST_OF_SETTINGS)
|
||||
|
||||
/** Set the settings from the profile (in the server configuration, many settings can be listed in one profile).
|
||||
* The profile can also be set using the `set` functions, like the `profile` setting.
|
||||
*/
|
||||
@ -6083,7 +6081,7 @@ void SettingsImpl::applyCompatibilitySetting(const String & compatibility_value)
|
||||
}
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
|
||||
Settings ## TYPE NAME = & Settings ## Impl :: NAME;
|
||||
Settings ## TYPE NAME = & SettingsImpl :: NAME;
|
||||
|
||||
namespace Setting
|
||||
{
|
||||
@ -6117,18 +6115,7 @@ bool Settings::operator==(const Settings & other) const
|
||||
return *impl == *other.impl;
|
||||
}
|
||||
|
||||
#define IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR(CLASS_NAME, TYPE) \
|
||||
const SettingField##TYPE & Settings::operator[](CLASS_NAME##TYPE t) const \
|
||||
{ \
|
||||
return impl.get()->*t; \
|
||||
} \
|
||||
SettingField##TYPE & Settings::operator[](CLASS_NAME##TYPE t) \
|
||||
{ \
|
||||
return impl.get()->*t; \
|
||||
}
|
||||
|
||||
COMMON_SETTINGS_SUPPORTED_TYPES(Settings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
#undef IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR
|
||||
|
||||
bool Settings::has(std::string_view name) const
|
||||
{
|
||||
|
@ -64,6 +64,12 @@ namespace ServerSetting
|
||||
extern const ServerSettingsUInt32 max_database_replicated_create_table_thread_pool_size;
|
||||
}
|
||||
|
||||
namespace DatabaseReplicatedSetting
|
||||
{
|
||||
extern const DatabaseReplicatedSettingsString collection_name;
|
||||
extern const DatabaseReplicatedSettingsFloat max_broken_tables_ratio;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NO_ZOOKEEPER;
|
||||
@ -141,8 +147,8 @@ DatabaseReplicated::DatabaseReplicated(
|
||||
if (zookeeper_path.front() != '/')
|
||||
zookeeper_path = "/" + zookeeper_path;
|
||||
|
||||
if (!db_settings.collection_name.value.empty())
|
||||
fillClusterAuthInfo(db_settings.collection_name.value, context_->getConfigRef());
|
||||
if (!db_settings[DatabaseReplicatedSetting::collection_name].value.empty())
|
||||
fillClusterAuthInfo(db_settings[DatabaseReplicatedSetting::collection_name].value, context_->getConfigRef());
|
||||
|
||||
replica_group_name = context_->getConfigRef().getString("replica_group_name", "");
|
||||
|
||||
@ -1220,7 +1226,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
|
||||
String db_name = getDatabaseName();
|
||||
String to_db_name = getDatabaseName() + BROKEN_TABLES_SUFFIX;
|
||||
String to_db_name_replicated = getDatabaseName() + BROKEN_REPLICATED_TABLES_SUFFIX;
|
||||
if (total_tables * db_settings.max_broken_tables_ratio < tables_to_detach.size())
|
||||
if (total_tables * db_settings[DatabaseReplicatedSetting::max_broken_tables_ratio] < tables_to_detach.size())
|
||||
throw Exception(ErrorCodes::DATABASE_REPLICATION_FAILED, "Too many tables to recreate: {} of {}", tables_to_detach.size(), total_tables);
|
||||
if (!tables_to_detach.empty())
|
||||
{
|
||||
|
@ -1,17 +1,60 @@
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Databases/DatabaseReplicatedSettings.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
#define LIST_OF_DATABASE_REPLICATED_SETTINGS(M, ALIAS) \
|
||||
M(Float, max_broken_tables_ratio, 1, "Do not recover replica automatically if the ratio of staled tables to all tables is greater", 0) \
|
||||
M(UInt64, max_replication_lag_to_enqueue, 50, "Replica will throw exception on attempt to execute query if its replication lag greater", 0) \
|
||||
M(UInt64, wait_entry_commited_timeout_sec, 3600, "Replicas will try to cancel query if timeout exceed, but initiator host has not executed it yet", 0) \
|
||||
M(String, collection_name, "", "A name of a collection defined in server's config where all info for cluster authentication is defined", 0) \
|
||||
M(Bool, check_consistency, true, "Check consistency of local metadata and metadata in Keeper, do replica recovery on inconsistency", 0) \
|
||||
M(UInt64, max_retries_before_automatic_recovery, 100, "Max number of attempts to execute a queue entry before marking replica as lost recovering it from snapshot (0 means infinite)", 0) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(DatabaseReplicatedSettingsTraits, LIST_OF_DATABASE_REPLICATED_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(DatabaseReplicatedSettingsTraits, LIST_OF_DATABASE_REPLICATED_SETTINGS)
|
||||
|
||||
struct DatabaseReplicatedSettingsImpl : public BaseSettings<DatabaseReplicatedSettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
|
||||
DatabaseReplicatedSettings##TYPE NAME = &DatabaseReplicatedSettingsImpl ::NAME;
|
||||
|
||||
namespace DatabaseReplicatedSetting
|
||||
{
|
||||
LIST_OF_DATABASE_REPLICATED_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
}
|
||||
|
||||
#undef INITIALIZE_SETTING_EXTERN
|
||||
|
||||
DatabaseReplicatedSettings::DatabaseReplicatedSettings() : impl(std::make_unique<DatabaseReplicatedSettingsImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
DatabaseReplicatedSettings::DatabaseReplicatedSettings(const DatabaseReplicatedSettings & settings)
|
||||
: impl(std::make_unique<DatabaseReplicatedSettingsImpl>(*settings.impl))
|
||||
{
|
||||
}
|
||||
|
||||
DatabaseReplicatedSettings::DatabaseReplicatedSettings(DatabaseReplicatedSettings && settings) noexcept
|
||||
: impl(std::make_unique<DatabaseReplicatedSettingsImpl>(std::move(*settings.impl)))
|
||||
{
|
||||
}
|
||||
|
||||
DatabaseReplicatedSettings::~DatabaseReplicatedSettings() = default;
|
||||
|
||||
DATABASE_REPLICATED_SETTINGS_SUPPORTED_TYPES(DatabaseReplicatedSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void DatabaseReplicatedSettings::loadFromQuery(ASTStorage & storage_def)
|
||||
{
|
||||
if (storage_def.settings)
|
||||
{
|
||||
applyChanges(storage_def.settings->changes);
|
||||
impl->applyChanges(storage_def.settings->changes);
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -1,27 +1,35 @@
|
||||
#pragma once
|
||||
#include <Core/Defines.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ASTStorage;
|
||||
struct DatabaseReplicatedSettingsImpl;
|
||||
|
||||
#define LIST_OF_DATABASE_REPLICATED_SETTINGS(M, ALIAS) \
|
||||
M(Float, max_broken_tables_ratio, 1, "Do not recover replica automatically if the ratio of staled tables to all tables is greater", 0) \
|
||||
M(UInt64, max_replication_lag_to_enqueue, 50, "Replica will throw exception on attempt to execute query if its replication lag greater", 0) \
|
||||
M(UInt64, wait_entry_commited_timeout_sec, 3600, "Replicas will try to cancel query if timeout exceed, but initiator host has not executed it yet", 0) \
|
||||
M(String, collection_name, "", "A name of a collection defined in server's config where all info for cluster authentication is defined", 0) \
|
||||
M(Bool, check_consistency, true, "Check consistency of local metadata and metadata in Keeper, do replica recovery on inconsistency", 0) \
|
||||
M(UInt64, max_retries_before_automatic_recovery, 100, "Max number of attempts to execute a queue entry before marking replica as lost recovering it from snapshot (0 means infinite)", 0) \
|
||||
/// List of available types supported in ReplicatedSettings object
|
||||
#define DATABASE_REPLICATED_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, Float) \
|
||||
M(CLASS_NAME, String) \
|
||||
M(CLASS_NAME, UInt64)
|
||||
|
||||
DATABASE_REPLICATED_SETTINGS_SUPPORTED_TYPES(DatabaseReplicatedSettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(DatabaseReplicatedSettingsTraits, LIST_OF_DATABASE_REPLICATED_SETTINGS)
|
||||
|
||||
|
||||
struct DatabaseReplicatedSettings : public BaseSettings<DatabaseReplicatedSettingsTraits>
|
||||
struct DatabaseReplicatedSettings
|
||||
{
|
||||
DatabaseReplicatedSettings();
|
||||
DatabaseReplicatedSettings(const DatabaseReplicatedSettings & settings);
|
||||
DatabaseReplicatedSettings(DatabaseReplicatedSettings && settings) noexcept;
|
||||
~DatabaseReplicatedSettings();
|
||||
|
||||
DATABASE_REPLICATED_SETTINGS_SUPPORTED_TYPES(DatabaseReplicatedSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void loadFromQuery(ASTStorage & storage_def);
|
||||
|
||||
private:
|
||||
std::unique_ptr<DatabaseReplicatedSettingsImpl> impl;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -18,6 +18,14 @@ namespace Setting
|
||||
extern const SettingsUInt64 database_replicated_initial_query_timeout_sec;
|
||||
}
|
||||
|
||||
namespace DatabaseReplicatedSetting
|
||||
{
|
||||
extern const DatabaseReplicatedSettingsBool check_consistency;
|
||||
extern const DatabaseReplicatedSettingsUInt64 max_replication_lag_to_enqueue;
|
||||
extern const DatabaseReplicatedSettingsUInt64 max_retries_before_automatic_recovery;
|
||||
extern const DatabaseReplicatedSettingsUInt64 wait_entry_commited_timeout_sec;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -63,8 +71,8 @@ bool DatabaseReplicatedDDLWorker::initializeMainThread()
|
||||
break;
|
||||
}
|
||||
|
||||
if (database->db_settings.max_retries_before_automatic_recovery &&
|
||||
database->db_settings.max_retries_before_automatic_recovery <= subsequent_errors_count)
|
||||
if (database->db_settings[DatabaseReplicatedSetting::max_retries_before_automatic_recovery]
|
||||
&& database->db_settings[DatabaseReplicatedSetting::max_retries_before_automatic_recovery] <= subsequent_errors_count)
|
||||
{
|
||||
String current_task_name;
|
||||
{
|
||||
@ -155,7 +163,7 @@ void DatabaseReplicatedDDLWorker::initializeReplication()
|
||||
|
||||
bool is_new_replica = our_log_ptr == 0;
|
||||
bool lost_according_to_log_ptr = our_log_ptr + logs_to_keep < max_log_ptr;
|
||||
bool lost_according_to_digest = database->db_settings.check_consistency && local_digest != digest;
|
||||
bool lost_according_to_digest = database->db_settings[DatabaseReplicatedSetting::check_consistency] && local_digest != digest;
|
||||
|
||||
if (is_new_replica || lost_according_to_log_ptr || lost_according_to_digest)
|
||||
{
|
||||
@ -306,7 +314,7 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr
|
||||
UInt32 our_log_ptr = getLogPointer();
|
||||
UInt32 max_log_ptr = parse<UInt32>(zookeeper->get(database->zookeeper_path + "/max_log_ptr"));
|
||||
|
||||
if (our_log_ptr + database->db_settings.max_replication_lag_to_enqueue < max_log_ptr)
|
||||
if (our_log_ptr + database->db_settings[DatabaseReplicatedSetting::max_replication_lag_to_enqueue] < max_log_ptr)
|
||||
throw Exception(ErrorCodes::NOT_A_LEADER, "Cannot enqueue query on this replica, "
|
||||
"because it has replication lag of {} queries. Try other replica.", max_log_ptr - our_log_ptr);
|
||||
|
||||
@ -399,7 +407,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na
|
||||
/// Query is not committed yet. We cannot just skip it and execute next one, because reordering may break replication.
|
||||
LOG_TRACE(log, "Waiting for initiator {} to commit or rollback entry {}", initiator_name, entry_path);
|
||||
constexpr size_t wait_time_ms = 1000;
|
||||
size_t max_iterations = database->db_settings.wait_entry_commited_timeout_sec;
|
||||
size_t max_iterations = database->db_settings[DatabaseReplicatedSetting::wait_entry_commited_timeout_sec];
|
||||
size_t iteration = 0;
|
||||
|
||||
while (!wait_committed_or_failed->tryWait(wait_time_ms))
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h>
|
||||
#include <Storages/PostgreSQL/MaterializedPostgreSQLSettings.h>
|
||||
|
||||
#if USE_LIBPQXX
|
||||
|
||||
@ -37,6 +38,11 @@ namespace Setting
|
||||
extern const SettingsUInt64 postgresql_connection_attempt_timeout;
|
||||
}
|
||||
|
||||
namespace MaterializedPostgreSQLSetting
|
||||
{
|
||||
extern const MaterializedPostgreSQLSettingsString materialized_postgresql_tables_list;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -362,7 +368,7 @@ void DatabaseMaterializedPostgreSQL::attachTable(ContextPtr context_, const Stri
|
||||
|
||||
try
|
||||
{
|
||||
auto tables_to_replicate = settings->materialized_postgresql_tables_list.value;
|
||||
auto tables_to_replicate = (*settings)[MaterializedPostgreSQLSetting::materialized_postgresql_tables_list].value;
|
||||
if (tables_to_replicate.empty())
|
||||
tables_to_replicate = getFormattedTablesList();
|
||||
|
||||
|
@ -5,7 +5,6 @@
|
||||
#if USE_LIBPQXX
|
||||
|
||||
#include <Storages/PostgreSQL/PostgreSQLReplicationHandler.h>
|
||||
#include <Storages/PostgreSQL/MaterializedPostgreSQLSettings.h>
|
||||
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
@ -18,6 +17,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct MaterializedPostgreSQLSettings;
|
||||
class PostgreSQLConnection;
|
||||
using PostgreSQLConnectionPtr = std::shared_ptr<PostgreSQLConnection>;
|
||||
|
||||
|
@ -158,21 +158,6 @@ S3::URI getS3URI(const Poco::Util::AbstractConfiguration & config, const std::st
|
||||
return uri;
|
||||
}
|
||||
|
||||
void checkS3Capabilities(
|
||||
S3ObjectStorage & storage, const S3Capabilities s3_capabilities, const String & name)
|
||||
{
|
||||
/// If `support_batch_delete` is turned on (default), check and possibly switch it off.
|
||||
if (s3_capabilities.support_batch_delete && !checkBatchRemove(storage))
|
||||
{
|
||||
LOG_WARNING(
|
||||
getLogger("S3ObjectStorage"),
|
||||
"Storage for disk {} does not support batch delete operations, "
|
||||
"so `s3_capabilities.support_batch_delete` was automatically turned off during the access check. "
|
||||
"To remove this message set `s3_capabilities.support_batch_delete` for the disk to `false`.",
|
||||
name);
|
||||
storage.setCapabilitiesSupportBatchDelete(false);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static std::string getEndpoint(
|
||||
@ -192,7 +177,7 @@ void registerS3ObjectStorage(ObjectStorageFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const ContextPtr & context,
|
||||
bool skip_access_check) -> ObjectStoragePtr
|
||||
bool /* skip_access_check */) -> ObjectStoragePtr
|
||||
{
|
||||
auto uri = getS3URI(config, config_prefix, context);
|
||||
auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix);
|
||||
@ -204,10 +189,6 @@ void registerS3ObjectStorage(ObjectStorageFactory & factory)
|
||||
auto object_storage = createObjectStorage<S3ObjectStorage>(
|
||||
ObjectStorageType::S3, config, config_prefix, std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name);
|
||||
|
||||
/// NOTE: should we still perform this check for clickhouse-disks?
|
||||
if (!skip_access_check)
|
||||
checkS3Capabilities(*dynamic_cast<S3ObjectStorage *>(object_storage.get()), s3_capabilities, name);
|
||||
|
||||
return object_storage;
|
||||
});
|
||||
}
|
||||
@ -221,7 +202,7 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const ContextPtr & context,
|
||||
bool skip_access_check) -> ObjectStoragePtr
|
||||
bool /* skip_access_check */) -> ObjectStoragePtr
|
||||
{
|
||||
/// send_metadata changes the filenames (includes revision), while
|
||||
/// s3_plain do not care about this, and expect that the file name
|
||||
@ -241,10 +222,6 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory)
|
||||
auto object_storage = std::make_shared<PlainObjectStorage<S3ObjectStorage>>(
|
||||
std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name);
|
||||
|
||||
/// NOTE: should we still perform this check for clickhouse-disks?
|
||||
if (!skip_access_check)
|
||||
checkS3Capabilities(*dynamic_cast<S3ObjectStorage *>(object_storage.get()), s3_capabilities, name);
|
||||
|
||||
return object_storage;
|
||||
});
|
||||
}
|
||||
@ -259,7 +236,7 @@ void registerS3PlainRewritableObjectStorage(ObjectStorageFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const ContextPtr & context,
|
||||
bool skip_access_check) -> ObjectStoragePtr
|
||||
bool /* skip_access_check */) -> ObjectStoragePtr
|
||||
{
|
||||
/// send_metadata changes the filenames (includes revision), while
|
||||
/// s3_plain_rewritable does not support file renaming.
|
||||
@ -277,10 +254,6 @@ void registerS3PlainRewritableObjectStorage(ObjectStorageFactory & factory)
|
||||
auto object_storage = std::make_shared<PlainRewritableObjectStorage<S3ObjectStorage>>(
|
||||
std::move(metadata_storage_metrics), std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name);
|
||||
|
||||
/// NOTE: should we still perform this check for clickhouse-disks?
|
||||
if (!skip_access_check)
|
||||
checkS3Capabilities(*dynamic_cast<S3ObjectStorage *>(object_storage.get()), s3_capabilities, name);
|
||||
|
||||
return object_storage;
|
||||
});
|
||||
}
|
||||
|
@ -2,8 +2,6 @@
|
||||
|
||||
#if USE_AWS_S3
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageMetadata.h>
|
||||
#include <Disks/ObjectStorages/S3/S3ObjectStorage.h>
|
||||
#include <Core/ServerUUID.h>
|
||||
#include <IO/S3/URI.h>
|
||||
|
||||
namespace DB
|
||||
@ -11,7 +9,6 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
ObjectStorageKeysGeneratorPtr getKeyGenerator(
|
||||
@ -65,58 +62,6 @@ ObjectStorageKeysGeneratorPtr getKeyGenerator(
|
||||
return createObjectStorageKeysGeneratorByTemplate(object_key_template);
|
||||
}
|
||||
|
||||
static String getServerUUID()
|
||||
{
|
||||
UUID server_uuid = ServerUUID::get();
|
||||
if (server_uuid == UUIDHelpers::Nil)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Server UUID is not initialized");
|
||||
return toString(server_uuid);
|
||||
}
|
||||
|
||||
bool checkBatchRemove(S3ObjectStorage & storage)
|
||||
{
|
||||
/// NOTE: Here we are going to write and later drop some key.
|
||||
/// We are using generateObjectKeyForPath() which returns random object key.
|
||||
/// That generated key is placed in a right directory where we should have write access.
|
||||
const String path = fmt::format("clickhouse_remove_objects_capability_{}", getServerUUID());
|
||||
const auto key = storage.generateObjectKeyForPath(path, {} /* key_prefix */);
|
||||
StoredObject object(key.serialize(), path);
|
||||
try
|
||||
{
|
||||
auto file = storage.writeObject(object, WriteMode::Rewrite);
|
||||
file->write("test", 4);
|
||||
file->finalize();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
try
|
||||
{
|
||||
storage.removeObject(object);
|
||||
}
|
||||
catch (...) // NOLINT(bugprone-empty-catch)
|
||||
{
|
||||
}
|
||||
/// We don't have write access, therefore no information about batch remove.
|
||||
return true;
|
||||
}
|
||||
try
|
||||
{
|
||||
/// Uses `DeleteObjects` request (batch delete).
|
||||
storage.removeObjects({object});
|
||||
return true;
|
||||
}
|
||||
catch (const Exception &)
|
||||
{
|
||||
try
|
||||
{
|
||||
storage.removeObject(object);
|
||||
}
|
||||
catch (...) // NOLINT(bugprone-empty-catch)
|
||||
{
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -16,9 +16,6 @@ ObjectStorageKeysGeneratorPtr getKeyGenerator(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix);
|
||||
|
||||
class S3ObjectStorage;
|
||||
bool checkBatchRemove(S3ObjectStorage & storage);
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -1,15 +0,0 @@
|
||||
#include <Disks/ObjectStorages/S3/S3Capabilities.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
S3Capabilities getCapabilitiesFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
|
||||
{
|
||||
return S3Capabilities
|
||||
{
|
||||
.support_batch_delete = config.getBool(config_prefix + ".support_batch_delete", true),
|
||||
.support_proxy = config.getBool(config_prefix + ".support_proxy", config.has(config_prefix + ".proxy")),
|
||||
};
|
||||
}
|
||||
|
||||
}
|
@ -1,26 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <string>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Supported/unsupported features by different S3 implementations
|
||||
/// Can be useful only for almost compatible with AWS S3 versions.
|
||||
struct S3Capabilities
|
||||
{
|
||||
/// Google S3 implementation doesn't support batch delete
|
||||
/// TODO: possibly we have to use Google SDK https://github.com/googleapis/google-cloud-cpp/tree/main/google/cloud/storage
|
||||
/// because looks like it misses some features:
|
||||
/// 1) batch delete (DeleteObjects)
|
||||
/// 2) upload part copy (UploadPartCopy)
|
||||
bool support_batch_delete{true};
|
||||
|
||||
/// Y.Cloud S3 implementation support proxy for connection
|
||||
bool support_proxy{false};
|
||||
};
|
||||
|
||||
S3Capabilities getCapabilitiesFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
|
||||
|
||||
}
|
@ -13,6 +13,7 @@
|
||||
#include <IO/ReadBufferFromS3.h>
|
||||
#include <IO/S3/getObjectInfo.h>
|
||||
#include <IO/S3/copyS3File.h>
|
||||
#include <IO/S3/deleteFileFromS3.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/threadPoolCallbackRunner.h>
|
||||
#include <Core/Settings.h>
|
||||
@ -29,7 +30,6 @@
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event S3DeleteObjects;
|
||||
extern const Event S3ListObjects;
|
||||
extern const Event DiskS3DeleteObjects;
|
||||
extern const Event DiskS3ListObjects;
|
||||
@ -73,20 +73,6 @@ void throwIfError(const Aws::Utils::Outcome<Result, Error> & response)
|
||||
}
|
||||
}
|
||||
|
||||
template <typename Result, typename Error>
|
||||
void throwIfUnexpectedError(const Aws::Utils::Outcome<Result, Error> & response, bool if_exists)
|
||||
{
|
||||
/// In this case even if absence of key may be ok for us,
|
||||
/// the log will be polluted with error messages from aws sdk.
|
||||
/// Looks like there is no way to suppress them.
|
||||
|
||||
if (!response.IsSuccess() && (!if_exists || !S3::isNotFoundError(response.GetError().GetErrorType())))
|
||||
{
|
||||
const auto & err = response.GetError();
|
||||
throw S3Exception(err.GetErrorType(), "{} (Code: {})", err.GetMessage(), static_cast<size_t>(err.GetErrorType()));
|
||||
}
|
||||
}
|
||||
|
||||
template <typename Result, typename Error>
|
||||
void logIfError(const Aws::Utils::Outcome<Result, Error> & response, std::function<String()> && msg)
|
||||
{
|
||||
@ -300,21 +286,11 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet
|
||||
|
||||
void S3ObjectStorage::removeObjectImpl(const StoredObject & object, bool if_exists)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::S3DeleteObjects);
|
||||
ProfileEvents::increment(ProfileEvents::DiskS3DeleteObjects);
|
||||
auto blob_storage_log = BlobStorageLogWriter::create(disk_name);
|
||||
|
||||
S3::DeleteObjectRequest request;
|
||||
request.SetBucket(uri.bucket);
|
||||
request.SetKey(object.remote_path);
|
||||
auto outcome = client.get()->DeleteObject(request);
|
||||
if (auto blob_storage_log = BlobStorageLogWriter::create(disk_name))
|
||||
blob_storage_log->addEvent(BlobStorageLogElement::EventType::Delete,
|
||||
uri.bucket, object.remote_path, object.local_path, object.bytes_size,
|
||||
outcome.IsSuccess() ? nullptr : &outcome.GetError());
|
||||
|
||||
throwIfUnexpectedError(outcome, if_exists);
|
||||
|
||||
LOG_DEBUG(log, "Object with path {} was removed from S3", object.remote_path);
|
||||
deleteFileFromS3(client.get(), uri.bucket, object.remote_path, if_exists,
|
||||
blob_storage_log, object.local_path, object.bytes_size,
|
||||
ProfileEvents::DiskS3DeleteObjects);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_exists)
|
||||
@ -322,59 +298,31 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e
|
||||
if (objects.empty())
|
||||
return;
|
||||
|
||||
if (!s3_capabilities.support_batch_delete)
|
||||
Strings keys;
|
||||
keys.reserve(objects.size());
|
||||
for (const auto & object : objects)
|
||||
keys.push_back(object.remote_path);
|
||||
|
||||
auto blob_storage_log = BlobStorageLogWriter::create(disk_name);
|
||||
Strings local_paths_for_blob_storage_log;
|
||||
std::vector<size_t> file_sizes_for_blob_storage_log;
|
||||
if (blob_storage_log)
|
||||
{
|
||||
local_paths_for_blob_storage_log.reserve(objects.size());
|
||||
file_sizes_for_blob_storage_log.reserve(objects.size());
|
||||
for (const auto & object : objects)
|
||||
removeObjectImpl(object, if_exists);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto settings_ptr = s3_settings.get();
|
||||
|
||||
size_t chunk_size_limit = settings_ptr->objects_chunk_size_to_delete;
|
||||
size_t current_position = 0;
|
||||
|
||||
auto blob_storage_log = BlobStorageLogWriter::create(disk_name);
|
||||
while (current_position < objects.size())
|
||||
{
|
||||
std::vector<Aws::S3::Model::ObjectIdentifier> current_chunk;
|
||||
String keys;
|
||||
size_t first_position = current_position;
|
||||
for (; current_position < objects.size() && current_chunk.size() < chunk_size_limit; ++current_position)
|
||||
{
|
||||
Aws::S3::Model::ObjectIdentifier obj;
|
||||
obj.SetKey(objects[current_position].remote_path);
|
||||
current_chunk.push_back(obj);
|
||||
|
||||
if (!keys.empty())
|
||||
keys += ", ";
|
||||
keys += objects[current_position].remote_path;
|
||||
}
|
||||
|
||||
Aws::S3::Model::Delete delkeys;
|
||||
delkeys.SetObjects(current_chunk);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::S3DeleteObjects);
|
||||
ProfileEvents::increment(ProfileEvents::DiskS3DeleteObjects);
|
||||
S3::DeleteObjectsRequest request;
|
||||
request.SetBucket(uri.bucket);
|
||||
request.SetDelete(delkeys);
|
||||
auto outcome = client.get()->DeleteObjects(request);
|
||||
|
||||
if (blob_storage_log)
|
||||
{
|
||||
const auto * outcome_error = outcome.IsSuccess() ? nullptr : &outcome.GetError();
|
||||
auto time_now = std::chrono::system_clock::now();
|
||||
for (size_t i = first_position; i < current_position; ++i)
|
||||
blob_storage_log->addEvent(BlobStorageLogElement::EventType::Delete,
|
||||
uri.bucket, objects[i].remote_path, objects[i].local_path, objects[i].bytes_size,
|
||||
outcome_error, time_now);
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Objects with paths [{}] were removed from S3", keys);
|
||||
throwIfUnexpectedError(outcome, if_exists);
|
||||
local_paths_for_blob_storage_log.push_back(object.local_path);
|
||||
file_sizes_for_blob_storage_log.push_back(object.bytes_size);
|
||||
}
|
||||
}
|
||||
|
||||
auto settings_ptr = s3_settings.get();
|
||||
|
||||
deleteFilesFromS3(client.get(), uri.bucket, keys, if_exists,
|
||||
s3_capabilities, settings_ptr->objects_chunk_size_to_delete,
|
||||
blob_storage_log, local_paths_for_blob_storage_log, file_sizes_for_blob_storage_log,
|
||||
ProfileEvents::DiskS3DeleteObjects);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::removeObject(const StoredObject & object)
|
||||
|
@ -5,8 +5,8 @@
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Disks/ObjectStorages/S3/S3Capabilities.h>
|
||||
#include <memory>
|
||||
#include <IO/S3/S3Capabilities.h>
|
||||
#include <IO/S3Settings.h>
|
||||
#include <Common/MultiVersion.h>
|
||||
#include <Common/ObjectStorageKeyGenerator.h>
|
||||
@ -148,8 +148,6 @@ public:
|
||||
|
||||
bool isRemote() const override { return true; }
|
||||
|
||||
void setCapabilitiesSupportBatchDelete(bool value) { s3_capabilities.support_batch_delete = value; }
|
||||
|
||||
std::unique_ptr<IObjectStorage> cloneObjectStorage(
|
||||
const std::string & new_namespace,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
|
47
src/IO/S3/S3Capabilities.cpp
Normal file
47
src/IO/S3/S3Capabilities.cpp
Normal file
@ -0,0 +1,47 @@
|
||||
#include <IO/S3/S3Capabilities.h>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
S3Capabilities::S3Capabilities(const S3Capabilities & src)
|
||||
: S3Capabilities(src.isBatchDeleteSupported(), src.support_proxy)
|
||||
{
|
||||
}
|
||||
|
||||
std::optional<bool> S3Capabilities::isBatchDeleteSupported() const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
return support_batch_delete;
|
||||
}
|
||||
|
||||
void S3Capabilities::setIsBatchDeleteSupported(bool support_batch_delete_)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
|
||||
if (support_batch_delete.has_value() && (support_batch_delete.value() != support_batch_delete_))
|
||||
{
|
||||
LOG_ERROR(getLogger("S3Capabilities"),
|
||||
"Got different results ({} vs {}) from checking if the cloud storage supports batch delete (DeleteObjects), "
|
||||
"the cloud storage API may be unstable",
|
||||
support_batch_delete.value(), support_batch_delete_);
|
||||
chassert(false && "Got different results from checking if the cloud storage supports batch delete");
|
||||
}
|
||||
|
||||
support_batch_delete = support_batch_delete_;
|
||||
}
|
||||
|
||||
S3Capabilities getCapabilitiesFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
|
||||
{
|
||||
std::optional<bool> support_batch_delete;
|
||||
if (config.has(config_prefix + ".support_batch_delete"))
|
||||
support_batch_delete = config.getBool(config_prefix + ".support_batch_delete");
|
||||
|
||||
bool support_proxy = config.getBool(config_prefix + ".support_proxy", config.has(config_prefix + ".proxy"));
|
||||
|
||||
return S3Capabilities{support_batch_delete, support_proxy};
|
||||
}
|
||||
|
||||
}
|
48
src/IO/S3/S3Capabilities.h
Normal file
48
src/IO/S3/S3Capabilities.h
Normal file
@ -0,0 +1,48 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/defines.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <mutex>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Supported/unsupported features by different S3 implementations
|
||||
/// Can be useful only for almost compatible with AWS S3 versions.
|
||||
class S3Capabilities
|
||||
{
|
||||
public:
|
||||
explicit S3Capabilities(std::optional<bool> support_batch_delete_ = {}, bool support_proxy_ = false)
|
||||
: support_proxy(support_proxy_), support_batch_delete(support_batch_delete_)
|
||||
{
|
||||
}
|
||||
|
||||
S3Capabilities(const S3Capabilities & src);
|
||||
|
||||
/// Google S3 implementation doesn't support batch delete
|
||||
/// TODO: possibly we have to use Google SDK https://github.com/googleapis/google-cloud-cpp/tree/main/google/cloud/storage
|
||||
/// because looks like it misses some features:
|
||||
/// 1) batch delete (DeleteObjects)
|
||||
/// 2) upload part copy (UploadPartCopy)
|
||||
/// If `isBatchDeleteSupported()` returns `nullopt` it means that it isn't clear yet if it's supported or not
|
||||
/// and should be detected automatically from responses of the cloud storage.
|
||||
std::optional<bool> isBatchDeleteSupported() const;
|
||||
void setIsBatchDeleteSupported(bool support_batch_delete_);
|
||||
|
||||
/// Y.Cloud S3 implementation support proxy for connection
|
||||
const bool support_proxy{false};
|
||||
|
||||
private:
|
||||
/// `support_batch_delete` is guarded by mutex because function deleteFilesFromS3() can update this field from another thread.
|
||||
/// If `support_batch_delete == nullopt` that means it's not clear yet if it's supported or not.
|
||||
std::optional<bool> support_batch_delete TSA_GUARDED_BY(mutex);
|
||||
|
||||
mutable std::mutex mutex;
|
||||
};
|
||||
|
||||
S3Capabilities getCapabilitiesFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
|
||||
|
||||
}
|
274
src/IO/S3/deleteFileFromS3.cpp
Normal file
274
src/IO/S3/deleteFileFromS3.cpp
Normal file
@ -0,0 +1,274 @@
|
||||
#include <IO/S3/deleteFileFromS3.h>
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <IO/S3/Client.h>
|
||||
#include <IO/S3/Requests.h>
|
||||
#include <IO/S3/BlobStorageLogWriter.h>
|
||||
#include <IO/S3/S3Capabilities.h>
|
||||
#include <IO/S3/getObjectInfo.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event S3DeleteObjects;
|
||||
}
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void deleteFileFromS3(
|
||||
const std::shared_ptr<const S3::Client> & s3_client,
|
||||
const String & bucket,
|
||||
const String & key,
|
||||
bool if_exists,
|
||||
BlobStorageLogWriterPtr blob_storage_log,
|
||||
const String & local_path_for_blob_storage_log,
|
||||
size_t file_size_for_blob_storage_log,
|
||||
std::optional<ProfileEvents::Event> profile_event)
|
||||
{
|
||||
S3::DeleteObjectRequest request;
|
||||
request.SetBucket(bucket);
|
||||
request.SetKey(key);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::S3DeleteObjects);
|
||||
if (profile_event && *profile_event != ProfileEvents::S3DeleteObjects)
|
||||
ProfileEvents::increment(*profile_event);
|
||||
|
||||
auto outcome = s3_client->DeleteObject(request);
|
||||
|
||||
auto log = getLogger("deleteFileFromS3");
|
||||
|
||||
if (blob_storage_log)
|
||||
{
|
||||
LOG_TRACE(log, "Writing Delete operation for blob {}", key);
|
||||
blob_storage_log->addEvent(BlobStorageLogElement::EventType::Delete,
|
||||
bucket, key,
|
||||
local_path_for_blob_storage_log, file_size_for_blob_storage_log,
|
||||
outcome.IsSuccess() ? nullptr : &outcome.GetError());
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_TRACE(log, "No blob storage log, not writing blob {}", key);
|
||||
}
|
||||
|
||||
if (outcome.IsSuccess())
|
||||
{
|
||||
LOG_INFO(log, "Object with path {} was removed from S3", key);
|
||||
}
|
||||
else if (if_exists && S3::isNotFoundError(outcome.GetError().GetErrorType()))
|
||||
{
|
||||
/// In this case even if absence of key may be ok for us, the log will be polluted with error messages from aws sdk.
|
||||
/// Looks like there is no way to suppress them.
|
||||
LOG_TRACE(log, "Object with path {} was skipped because it didn't exist", key);
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & err = outcome.GetError();
|
||||
throw S3Exception(err.GetErrorType(), "{} (Code: {}) while removing object with path {} from S3",
|
||||
err.GetMessage(), static_cast<size_t>(err.GetErrorType()), key);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void deleteFilesFromS3(
|
||||
const std::shared_ptr<const S3::Client> & s3_client,
|
||||
const String & bucket,
|
||||
const Strings & keys,
|
||||
bool if_exists,
|
||||
S3Capabilities & s3_capabilities,
|
||||
size_t batch_size,
|
||||
BlobStorageLogWriterPtr blob_storage_log,
|
||||
const Strings & local_paths_for_blob_storage_log,
|
||||
const std::vector<size_t> & file_sizes_for_blob_storage_log,
|
||||
std::optional<ProfileEvents::Event> profile_event)
|
||||
{
|
||||
chassert(local_paths_for_blob_storage_log.empty() || (local_paths_for_blob_storage_log.size() == keys.size()));
|
||||
chassert(file_sizes_for_blob_storage_log.empty() || (file_sizes_for_blob_storage_log.size() == keys.size()));
|
||||
|
||||
if (keys.empty())
|
||||
return; /// Nothing to delete.
|
||||
|
||||
/// We're trying batch delete (DeleteObjects) first.
|
||||
bool try_batch_delete = true;
|
||||
{
|
||||
if (keys.size() == 1)
|
||||
try_batch_delete = false; /// We're deleting one file - there is no need for batch delete.
|
||||
else if (batch_size < 2)
|
||||
try_batch_delete = false; /// Can't do batch delete with such small batches.
|
||||
else if (auto support_batch_delete = s3_capabilities.isBatchDeleteSupported();
|
||||
support_batch_delete.has_value() && !support_batch_delete.value())
|
||||
try_batch_delete = false; /// Support for batch delete is disabled.
|
||||
}
|
||||
|
||||
auto log = getLogger("deleteFileFromS3");
|
||||
const String empty_string;
|
||||
|
||||
if (try_batch_delete)
|
||||
{
|
||||
bool need_retry_with_plain_delete_object = false;
|
||||
size_t current_position = 0;
|
||||
|
||||
while (current_position < keys.size())
|
||||
{
|
||||
std::vector<Aws::S3::Model::ObjectIdentifier> current_chunk;
|
||||
String comma_separated_keys;
|
||||
size_t first_position = current_position;
|
||||
for (; current_position < keys.size() && current_chunk.size() < batch_size; ++current_position)
|
||||
{
|
||||
Aws::S3::Model::ObjectIdentifier obj;
|
||||
obj.SetKey(keys[current_position]);
|
||||
current_chunk.push_back(obj);
|
||||
|
||||
if (!comma_separated_keys.empty())
|
||||
comma_separated_keys += ", ";
|
||||
comma_separated_keys += keys[current_position];
|
||||
}
|
||||
|
||||
Aws::S3::Model::Delete delkeys;
|
||||
delkeys.SetObjects(current_chunk);
|
||||
delkeys.SetQuiet(true);
|
||||
|
||||
S3::DeleteObjectsRequest request;
|
||||
request.SetBucket(bucket);
|
||||
request.SetDelete(delkeys);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::S3DeleteObjects);
|
||||
if (profile_event && *profile_event != ProfileEvents::S3DeleteObjects)
|
||||
ProfileEvents::increment(*profile_event);
|
||||
|
||||
auto outcome = s3_client->DeleteObjects(request);
|
||||
|
||||
if (blob_storage_log)
|
||||
{
|
||||
const auto * outcome_error = outcome.IsSuccess() ? nullptr : &outcome.GetError();
|
||||
auto time_now = std::chrono::system_clock::now();
|
||||
LOG_TRACE(log, "Writing Delete operation for blobs [{}]", comma_separated_keys);
|
||||
for (size_t i = first_position; i < current_position; ++i)
|
||||
{
|
||||
const String & local_path_for_blob_storage_log = (i < local_paths_for_blob_storage_log.size()) ? local_paths_for_blob_storage_log[i] : empty_string;
|
||||
size_t file_size_for_blob_storage_log = (i < file_sizes_for_blob_storage_log.size()) ? file_sizes_for_blob_storage_log[i] : 0;
|
||||
|
||||
blob_storage_log->addEvent(BlobStorageLogElement::EventType::Delete,
|
||||
bucket, keys[i],
|
||||
local_path_for_blob_storage_log, file_size_for_blob_storage_log,
|
||||
outcome_error, time_now);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_TRACE(log, "No blob storage log, not writing blobs [{}]", comma_separated_keys);
|
||||
}
|
||||
|
||||
if (outcome.IsSuccess())
|
||||
{
|
||||
/// DeleteObjects succeeded, that means some objects were removed (but maybe not all the objects).
|
||||
/// Multiple threads can call deleteFilesFromS3() with a reference to the same `s3_capabilities`,
|
||||
/// and the following line doesn't cause a race because `s3_capabilities` is protected with mutex.
|
||||
s3_capabilities.setIsBatchDeleteSupported(true);
|
||||
|
||||
const auto & errors = outcome.GetResult().GetErrors();
|
||||
if (errors.empty())
|
||||
{
|
||||
/// All the objects were removed.
|
||||
LOG_INFO(log, "Objects with paths [{}] were removed from S3", comma_separated_keys);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Mixed success/error response - some objects were removed, and some were not.
|
||||
/// We need to extract more detailed information from the outcome.
|
||||
std::unordered_set<std::string_view> removed_keys{keys.begin(), keys.end()};
|
||||
String not_found_keys;
|
||||
std::exception_ptr other_error;
|
||||
|
||||
for (const auto & err : errors)
|
||||
{
|
||||
removed_keys.erase(err.GetKey());
|
||||
auto error_type = static_cast<Aws::S3::S3Errors>(Aws::S3::S3ErrorMapper::GetErrorForName(err.GetCode().c_str()).GetErrorType());
|
||||
if (if_exists && S3::isNotFoundError(error_type))
|
||||
{
|
||||
if (not_found_keys.empty())
|
||||
not_found_keys += ", ";
|
||||
not_found_keys += err.GetKey();
|
||||
}
|
||||
else if (!other_error)
|
||||
{
|
||||
other_error = std::make_exception_ptr(
|
||||
S3Exception{error_type, "{} (Code: {}) while removing object with path {} from S3",
|
||||
err.GetMessage(), err.GetCode(), err.GetKey()});
|
||||
}
|
||||
}
|
||||
|
||||
if (!removed_keys.empty())
|
||||
{
|
||||
String removed_keys_comma_separated;
|
||||
for (const auto & key : removed_keys)
|
||||
{
|
||||
if (!removed_keys_comma_separated.empty())
|
||||
removed_keys_comma_separated += ", ";
|
||||
removed_keys_comma_separated += key;
|
||||
}
|
||||
LOG_INFO(log, "Objects with paths [{}] were removed from S3", removed_keys_comma_separated);
|
||||
}
|
||||
|
||||
if (!not_found_keys.empty())
|
||||
{
|
||||
/// In this case even if absence of key may be ok for us, the log will be polluted with error messages from aws sdk.
|
||||
/// Looks like there is no way to suppress them.
|
||||
LOG_TRACE(log, "Object with paths [{}] were skipped because they didn't exist", not_found_keys);
|
||||
}
|
||||
|
||||
if (other_error)
|
||||
std::rethrow_exception(other_error);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// DeleteObjects didn't succeed, that means either a) this operation isn't supported at all;
|
||||
/// or b) all the objects didn't exist; or c) some failure occurred.
|
||||
const auto & err = outcome.GetError();
|
||||
if ((err.GetExceptionName() == "InvalidRequest") || (err.GetExceptionName() == "InvalidArgument")
|
||||
|| (err.GetExceptionName() == "NotImplemented"))
|
||||
{
|
||||
LOG_TRACE(log, "DeleteObjects is not supported: {} (Code: {}). Retrying with plain DeleteObject.",
|
||||
err.GetMessage(), static_cast<size_t>(err.GetErrorType()));
|
||||
/// Multiple threads can call deleteFilesFromS3() with a reference to the same `s3_capabilities`,
|
||||
/// and the following line doesn't cause a race because `s3_capabilities` is protected with mutex.
|
||||
s3_capabilities.setIsBatchDeleteSupported(false);
|
||||
need_retry_with_plain_delete_object = true;
|
||||
break;
|
||||
}
|
||||
|
||||
if (if_exists && S3::isNotFoundError(err.GetErrorType()))
|
||||
{
|
||||
LOG_TRACE(log, "Object with paths [{}] were skipped because they didn't exist", comma_separated_keys);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw S3Exception(err.GetErrorType(), "{} (Code: {}) while removing objects with paths [{}] from S3",
|
||||
err.GetMessage(), static_cast<size_t>(err.GetErrorType()), comma_separated_keys);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!need_retry_with_plain_delete_object)
|
||||
return;
|
||||
}
|
||||
|
||||
/// Batch delete (DeleteObjects) isn't supported so we'll delete all the files sequentially.
|
||||
for (size_t i = 0; i != keys.size(); ++i)
|
||||
{
|
||||
const String & local_path_for_blob_storage_log = (i < local_paths_for_blob_storage_log.size()) ? local_paths_for_blob_storage_log[i] : empty_string;
|
||||
size_t file_size_for_blob_storage_log = (i < file_sizes_for_blob_storage_log.size()) ? file_sizes_for_blob_storage_log[i] : 0;
|
||||
|
||||
deleteFileFromS3(s3_client, bucket, keys[i], if_exists,
|
||||
blob_storage_log, local_path_for_blob_storage_log, file_size_for_blob_storage_log,
|
||||
profile_event);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
51
src/IO/S3/deleteFileFromS3.h
Normal file
51
src/IO/S3/deleteFileFromS3.h
Normal file
@ -0,0 +1,51 @@
|
||||
#pragma once
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Core/Types.h>
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace S3
|
||||
{
|
||||
class Client;
|
||||
}
|
||||
|
||||
class S3Capabilities;
|
||||
class BlobStorageLogWriter;
|
||||
using BlobStorageLogWriterPtr = std::shared_ptr<BlobStorageLogWriter>;
|
||||
|
||||
|
||||
/// Deletes one file from S3.
|
||||
void deleteFileFromS3(
|
||||
const std::shared_ptr<const S3::Client> & s3_client,
|
||||
const String & bucket,
|
||||
const String & key,
|
||||
bool if_exists = false,
|
||||
BlobStorageLogWriterPtr blob_storage_log = nullptr,
|
||||
const String & local_path_for_blob_storage_log = {},
|
||||
size_t file_size_for_blob_storage_log = 0,
|
||||
std::optional<ProfileEvents::Event> profile_event = std::nullopt);
|
||||
|
||||
/// Deletes multiple files from S3 using batch requests when it's possible.
|
||||
void deleteFilesFromS3(
|
||||
const std::shared_ptr<const S3::Client> & s3_client,
|
||||
const String & bucket,
|
||||
const Strings & keys,
|
||||
bool if_exists,
|
||||
S3Capabilities & s3_capabilities,
|
||||
size_t batch_size = 1000,
|
||||
BlobStorageLogWriterPtr blob_storage_log = nullptr,
|
||||
const Strings & local_paths_for_blob_storage_log = {},
|
||||
const std::vector<size_t> & file_sizes_for_blob_storage_log = {},
|
||||
std::optional<ProfileEvents::Event> profile_event = std::nullopt);
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -66,7 +66,8 @@ namespace
|
||||
|
||||
bool isNotFoundError(Aws::S3::S3Errors error)
|
||||
{
|
||||
return error == Aws::S3::S3Errors::RESOURCE_NOT_FOUND || error == Aws::S3::S3Errors::NO_SUCH_KEY;
|
||||
return error == Aws::S3::S3Errors::RESOURCE_NOT_FOUND || error == Aws::S3::S3Errors::NO_SUCH_KEY
|
||||
|| error == Aws::S3::S3Errors::NO_SUCH_BUCKET;
|
||||
}
|
||||
|
||||
ObjectInfo getObjectInfo(
|
||||
|
@ -332,15 +332,20 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
|
||||
const auto * literal = arguments->children[0]->as<ASTLiteral>();
|
||||
const auto * function = arguments->children[0]->as<ASTFunction>();
|
||||
const auto * subquery = arguments->children[0]->as<ASTSubquery>();
|
||||
bool is_tuple = literal && literal->value.getType() == Field::Types::Tuple;
|
||||
/// Do not add parentheses for tuple literal, otherwise extra parens will be added `-((3, 7, 3), 1)` -> `-(((3, 7, 3), 1))`
|
||||
bool literal_need_parens = literal && !is_tuple;
|
||||
bool is_tuple = (literal && literal->value.getType() == Field::Types::Tuple)
|
||||
|| (function && function->name == "tuple" && function->arguments && function->arguments->children.size() > 1);
|
||||
bool is_array = (literal && literal->value.getType() == Field::Types::Array)
|
||||
|| (function && function->name == "array");
|
||||
|
||||
/// Do not add parentheses for tuple and array literal, otherwise extra parens will be added `-((3, 7, 3), 1)` -> `-(((3, 7, 3), 1))`, `-[1]` -> `-([1])`
|
||||
bool literal_need_parens = literal && !is_tuple && !is_array;
|
||||
|
||||
/// Negate always requires parentheses, otherwise -(-1) will be printed as --1
|
||||
/// Also extra parentheses are needed for subqueries, because NOT can be parsed as a function:
|
||||
/// Also extra parentheses are needed for subqueries and tuple, because NOT can be parsed as a function:
|
||||
/// not(SELECT 1) cannot be parsed, while not((SELECT 1)) can.
|
||||
/// not((1, 2, 3)) is a function of one argument, while not(1, 2, 3) is a function of three arguments.
|
||||
bool inside_parens = (name == "negate" && (literal_need_parens || (function && function->name == "negate")))
|
||||
|| (subquery && name == "not");
|
||||
|| (subquery && name == "not") || (is_tuple && name == "not");
|
||||
|
||||
/// We DO need parentheses around a single literal
|
||||
/// For example, SELECT (NOT 0) + (NOT 0) cannot be transformed into SELECT NOT 0 + NOT 0, since
|
||||
|
@ -32,12 +32,35 @@
|
||||
# include <IO/copyData.h>
|
||||
# include <Interpreters/castColumn.h>
|
||||
# include <Storages/MergeTree/KeyCondition.h>
|
||||
# include <boost/algorithm/string/case_conv.hpp>
|
||||
# include <orc/MemoryPool.hh>
|
||||
# include <Common/Allocator.h>
|
||||
# include <Common/FieldVisitorsAccurateComparison.h>
|
||||
# include "ArrowBufferedStreams.h"
|
||||
|
||||
# include <Common/MemorySanitizer.h>
|
||||
# include <orc/Vector.hh>
|
||||
|
||||
# include "ArrowBufferedStreams.h"
|
||||
|
||||
# include <boost/algorithm/string/case_conv.hpp>
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class MemoryPool : public orc::MemoryPool
|
||||
{
|
||||
public:
|
||||
char * malloc(uint64_t size) override
|
||||
{
|
||||
auto * ptr = ::malloc(size);
|
||||
/// For nullable columns some of the values will not be initialized.
|
||||
__msan_unpoison(ptr, size);
|
||||
return static_cast<char *>(ptr);
|
||||
}
|
||||
|
||||
void free(char * p) override { ::free(p); }
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -724,12 +747,17 @@ buildORCSearchArgument(const KeyCondition & key_condition, const Block & header,
|
||||
}
|
||||
|
||||
static void getFileReader(
|
||||
ReadBuffer & in, std::unique_ptr<orc::Reader> & file_reader, const FormatSettings & format_settings, std::atomic<int> & is_stopped)
|
||||
ReadBuffer & in,
|
||||
std::unique_ptr<orc::Reader> & file_reader,
|
||||
orc::MemoryPool & pool,
|
||||
const FormatSettings & format_settings,
|
||||
std::atomic<int> & is_stopped)
|
||||
{
|
||||
if (is_stopped)
|
||||
return;
|
||||
|
||||
orc::ReaderOptions options;
|
||||
options.setMemoryPool(pool);
|
||||
auto input_stream = asORCInputStream(in, format_settings, is_stopped);
|
||||
file_reader = orc::createReader(std::move(input_stream), options);
|
||||
}
|
||||
@ -875,6 +903,7 @@ static void updateIncludeTypeIds(
|
||||
|
||||
NativeORCBlockInputFormat::NativeORCBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_)
|
||||
: IInputFormat(std::move(header_), &in_)
|
||||
, memory_pool(std::make_unique<MemoryPool>())
|
||||
, block_missing_values(getPort().getHeader().columns())
|
||||
, format_settings(format_settings_)
|
||||
, skip_stripes(format_settings.orc.skip_stripes)
|
||||
@ -883,7 +912,7 @@ NativeORCBlockInputFormat::NativeORCBlockInputFormat(ReadBuffer & in_, Block hea
|
||||
|
||||
void NativeORCBlockInputFormat::prepareFileReader()
|
||||
{
|
||||
getFileReader(*in, file_reader, format_settings, is_stopped);
|
||||
getFileReader(*in, file_reader, *memory_pool, format_settings, is_stopped);
|
||||
if (is_stopped)
|
||||
return;
|
||||
|
||||
@ -1027,7 +1056,8 @@ NamesAndTypesList NativeORCSchemaReader::readSchema()
|
||||
{
|
||||
std::unique_ptr<orc::Reader> file_reader;
|
||||
std::atomic<int> is_stopped = 0;
|
||||
getFileReader(in, file_reader, format_settings, is_stopped);
|
||||
MemoryPool memory_pool;
|
||||
getFileReader(in, file_reader, memory_pool, format_settings, is_stopped);
|
||||
|
||||
|
||||
const auto & schema = file_reader->getType();
|
||||
|
@ -8,6 +8,7 @@
|
||||
# include <Processors/Formats/ISchemaReader.h>
|
||||
# include <Storages/MergeTree/KeyCondition.h>
|
||||
# include <boost/algorithm/string.hpp>
|
||||
# include <orc/MemoryPool.hh>
|
||||
# include <orc/OrcFile.hh>
|
||||
|
||||
namespace DB
|
||||
@ -70,6 +71,8 @@ private:
|
||||
void prepareFileReader();
|
||||
bool prepareStripeReader();
|
||||
|
||||
std::unique_ptr<orc::MemoryPool> memory_pool;
|
||||
|
||||
std::unique_ptr<orc::Reader> file_reader;
|
||||
std::unique_ptr<orc::RowReader> stripe_reader;
|
||||
std::unique_ptr<ORCColumnToCHColumn> orc_column_to_ch_column;
|
||||
|
@ -1,8 +1,57 @@
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Storages/MaterializedView/RefreshSettings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
#define LIST_OF_REFRESH_SETTINGS(M, ALIAS) \
|
||||
M(Int64, refresh_retries, 2, "How many times to retry refresh query if it fails. If all attempts fail, wait for the next refresh time according to schedule. 0 to disable retries. -1 for infinite retries.", 0) \
|
||||
M(UInt64, refresh_retry_initial_backoff_ms, 100, "Delay before the first retry if refresh query fails (if refresh_retries setting is not zero). Each subsequent retry doubles the delay, up to refresh_retry_max_backoff_ms.", 0) \
|
||||
M(UInt64, refresh_retry_max_backoff_ms, 60'000, "Limit on the exponential growth of delay between refresh attempts, if they keep failing and refresh_retries is positive.", 0) \
|
||||
M(Bool, all_replicas, /* do not change or existing tables will break */ false, "If the materialized view is in a Replicated database, and APPEND is enabled, this flag controls whether all replicas or one replica will refresh.", 0) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(RefreshSettingsTraits, LIST_OF_REFRESH_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(RefreshSettingsTraits, LIST_OF_REFRESH_SETTINGS)
|
||||
|
||||
struct RefreshSettingsImpl : public BaseSettings<RefreshSettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) RefreshSettings##TYPE NAME = &RefreshSettingsImpl ::NAME;
|
||||
|
||||
namespace RefreshSetting
|
||||
{
|
||||
LIST_OF_REFRESH_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
}
|
||||
|
||||
#undef INITIALIZE_SETTING_EXTERN
|
||||
|
||||
RefreshSettings::RefreshSettings() : impl(std::make_unique<RefreshSettingsImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
RefreshSettings::RefreshSettings(const RefreshSettings & settings) : impl(std::make_unique<RefreshSettingsImpl>(*settings.impl))
|
||||
{
|
||||
}
|
||||
|
||||
RefreshSettings::RefreshSettings(RefreshSettings && settings) noexcept
|
||||
: impl(std::make_unique<RefreshSettingsImpl>(std::move(*settings.impl)))
|
||||
{
|
||||
}
|
||||
|
||||
RefreshSettings::~RefreshSettings() = default;
|
||||
|
||||
RefreshSettings & RefreshSettings::operator=(const RefreshSettings & other)
|
||||
{
|
||||
*impl = *other.impl;
|
||||
return *this;
|
||||
}
|
||||
|
||||
REFRESH_SETTINGS_SUPPORTED_TYPES(RefreshSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void RefreshSettings::applyChanges(const SettingsChanges & changes)
|
||||
{
|
||||
impl->applyChanges(changes);
|
||||
}
|
||||
}
|
||||
|
@ -1,18 +1,35 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct RefreshSettingsImpl;
|
||||
class SettingsChanges;
|
||||
|
||||
#define LIST_OF_REFRESH_SETTINGS(M, ALIAS) \
|
||||
M(Int64, refresh_retries, 2, "How many times to retry refresh query if it fails. If all attempts fail, wait for the next refresh time according to schedule. 0 to disable retries. -1 for infinite retries.", 0) \
|
||||
M(UInt64, refresh_retry_initial_backoff_ms, 100, "Delay before the first retry if refresh query fails (if refresh_retries setting is not zero). Each subsequent retry doubles the delay, up to refresh_retry_max_backoff_ms.", 0) \
|
||||
M(UInt64, refresh_retry_max_backoff_ms, 60'000, "Limit on the exponential growth of delay between refresh attempts, if they keep failing and refresh_retries is positive.", 0) \
|
||||
M(Bool, all_replicas, /* do not change or existing tables will break */ false, "If the materialized view is in a Replicated database, and APPEND is enabled, this flag controls whether all replicas or one replica will refresh.", 0) \
|
||||
/// List of available types supported in RabbitMQSettings object
|
||||
#define REFRESH_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, Int64) \
|
||||
M(CLASS_NAME, UInt64)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(RefreshSettingsTraits, LIST_OF_REFRESH_SETTINGS)
|
||||
REFRESH_SETTINGS_SUPPORTED_TYPES(RefreshSettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
struct RefreshSettings : public BaseSettings<RefreshSettingsTraits> {};
|
||||
struct RefreshSettings
|
||||
{
|
||||
RefreshSettings();
|
||||
RefreshSettings(const RefreshSettings & settings);
|
||||
RefreshSettings(RefreshSettings && settings) noexcept;
|
||||
~RefreshSettings();
|
||||
|
||||
RefreshSettings & operator=(const RefreshSettings & other);
|
||||
|
||||
REFRESH_SETTINGS_SUPPORTED_TYPES(RefreshSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void applyChanges(const SettingsChanges & changes);
|
||||
|
||||
private:
|
||||
std::unique_ptr<RefreshSettingsImpl> impl;
|
||||
};
|
||||
}
|
||||
|
@ -38,6 +38,14 @@ namespace ServerSetting
|
||||
extern const ServerSettingsString default_replica_path;
|
||||
}
|
||||
|
||||
namespace RefreshSetting
|
||||
{
|
||||
extern const RefreshSettingsBool all_replicas;
|
||||
extern const RefreshSettingsInt64 refresh_retries;
|
||||
extern const RefreshSettingsUInt64 refresh_retry_initial_backoff_ms;
|
||||
extern const RefreshSettingsUInt64 refresh_retry_max_backoff_ms;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -187,7 +195,7 @@ void RefreshTask::checkAlterIsPossible(const DB::ASTRefreshStrategy & new_strate
|
||||
RefreshSettings s;
|
||||
if (new_strategy.settings)
|
||||
s.applyChanges(new_strategy.settings->changes);
|
||||
if (s.all_replicas != refresh_settings.all_replicas)
|
||||
if (s[RefreshSetting::all_replicas] != refresh_settings[RefreshSetting::all_replicas])
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Altering setting 'all_replicas' is not supported.");
|
||||
if (new_strategy.append != refresh_append)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Adding or removing APPEND is not supported.");
|
||||
@ -456,7 +464,7 @@ void RefreshTask::refreshTask()
|
||||
else
|
||||
{
|
||||
error_message = getCurrentExceptionMessage(true);
|
||||
LOG_ERROR(log, "{}: Refresh failed (attempt {}/{}): {}", view->getStorageID().getFullTableName(), start_znode.attempt_number, refresh_settings.refresh_retries + 1, error_message);
|
||||
LOG_ERROR(log, "{}: Refresh failed (attempt {}/{}): {}", view->getStorageID().getFullTableName(), start_znode.attempt_number, refresh_settings[RefreshSetting::refresh_retries] + 1, error_message);
|
||||
}
|
||||
}
|
||||
|
||||
@ -676,10 +684,10 @@ static std::chrono::milliseconds backoff(Int64 retry_idx, const RefreshSettings
|
||||
UInt64 delay_ms;
|
||||
UInt64 multiplier = UInt64(1) << std::min(retry_idx, Int64(62));
|
||||
/// Overflow check: a*b <= c iff a <= c/b iff a <= floor(c/b).
|
||||
if (refresh_settings.refresh_retry_initial_backoff_ms <= refresh_settings.refresh_retry_max_backoff_ms / multiplier)
|
||||
delay_ms = refresh_settings.refresh_retry_initial_backoff_ms * multiplier;
|
||||
if (refresh_settings[RefreshSetting::refresh_retry_initial_backoff_ms] <= refresh_settings[RefreshSetting::refresh_retry_max_backoff_ms] / multiplier)
|
||||
delay_ms = refresh_settings[RefreshSetting::refresh_retry_initial_backoff_ms] * multiplier;
|
||||
else
|
||||
delay_ms = refresh_settings.refresh_retry_max_backoff_ms;
|
||||
delay_ms = refresh_settings[RefreshSetting::refresh_retry_max_backoff_ms];
|
||||
return std::chrono::milliseconds(delay_ms);
|
||||
}
|
||||
|
||||
@ -687,7 +695,7 @@ std::tuple<std::chrono::system_clock::time_point, std::chrono::sys_seconds, Refr
|
||||
RefreshTask::determineNextRefreshTime(std::chrono::sys_seconds now)
|
||||
{
|
||||
auto znode = coordination.root_znode;
|
||||
if (refresh_settings.refresh_retries >= 0 && znode.attempt_number > refresh_settings.refresh_retries)
|
||||
if (refresh_settings[RefreshSetting::refresh_retries] >= 0 && znode.attempt_number > refresh_settings[RefreshSetting::refresh_retries])
|
||||
{
|
||||
/// Skip to the next scheduled refresh, as if a refresh succeeded.
|
||||
znode.last_completed_timeslot = refresh_schedule.timeslotForCompletedRefresh(znode.last_completed_timeslot, znode.last_attempt_time, znode.last_attempt_time, false);
|
||||
|
@ -42,6 +42,7 @@ namespace ErrorCodes
|
||||
extern const int INCORRECT_DATA;
|
||||
extern const int INCORRECT_NUMBER_OF_COLUMNS;
|
||||
extern const int INCORRECT_QUERY;
|
||||
extern const int INVALID_SETTING_VALUE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
@ -110,7 +111,7 @@ USearchIndexWithSerialization::USearchIndexWithSerialization(
|
||||
{
|
||||
USearchIndex::metric_t metric(dimensions, metric_kind, scalar_kind);
|
||||
|
||||
unum::usearch::index_dense_config_t config(usearch_hnsw_params.connectivity, usearch_hnsw_params.expansion_add, unum::usearch::default_expansion_search());
|
||||
unum::usearch::index_dense_config_t config(usearch_hnsw_params.connectivity, usearch_hnsw_params.expansion_add, default_expansion_search);
|
||||
config.enable_key_lookups = false; /// we don't do row-to-vector lookups
|
||||
|
||||
auto result = USearchIndex::make(metric, config);
|
||||
@ -407,6 +408,9 @@ MergeTreeIndexConditionVectorSimilarity::MergeTreeIndexConditionVectorSimilarity
|
||||
, metric_kind(metric_kind_)
|
||||
, expansion_search(context->getSettingsRef()[Setting::hnsw_candidate_list_size_for_search])
|
||||
{
|
||||
if (expansion_search == 0)
|
||||
throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Setting 'hnsw_candidate_list_size_for_search' must not be 0");
|
||||
|
||||
}
|
||||
|
||||
bool MergeTreeIndexConditionVectorSimilarity::mayBeTrueOnGranule(MergeTreeIndexGranulePtr) const
|
||||
@ -447,7 +451,7 @@ std::vector<UInt64> MergeTreeIndexConditionVectorSimilarity::calculateApproximat
|
||||
/// synchronize index access, see https://github.com/unum-cloud/usearch/issues/500. As a workaround, we extended USearch' search method
|
||||
/// to accept a custom expansion_add setting. The config value is only used on the fly, i.e. not persisted in the index.
|
||||
|
||||
auto search_result = index->search(reference_vector.data(), limit, USearchIndex::any_thread(), false, (expansion_search == 0) ? unum::usearch::default_expansion_search() : expansion_search);
|
||||
auto search_result = index->search(reference_vector.data(), limit, USearchIndex::any_thread(), false, expansion_search);
|
||||
if (!search_result)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Could not search in vector similarity index. Error: {}", String(search_result.error.release()));
|
||||
|
||||
@ -558,7 +562,7 @@ void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* atta
|
||||
/// Call Usearch's own parameter validation method for HNSW-specific parameters
|
||||
UInt64 connectivity = index.arguments[3].safeGet<UInt64>();
|
||||
UInt64 expansion_add = index.arguments[4].safeGet<UInt64>();
|
||||
UInt64 expansion_search = unum::usearch::default_expansion_search();
|
||||
UInt64 expansion_search = default_expansion_search;
|
||||
|
||||
unum::usearch::index_dense_config_t config(connectivity, expansion_add, expansion_search);
|
||||
if (auto error = config.validate(); error)
|
||||
|
@ -11,10 +11,18 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Defaults for HNSW parameters. Instead of using the default parameters provided by USearch (default_connectivity(),
|
||||
/// default_expansion_add(), default_expansion_search()), we experimentally came up with our own default parameters. They provide better
|
||||
/// trade-offs with regards to index construction time, search precision and queries-per-second (speed).
|
||||
static constexpr size_t default_connectivity = 32;
|
||||
static constexpr size_t default_expansion_add = 128;
|
||||
static constexpr size_t default_expansion_search = 256;
|
||||
|
||||
/// Parameters for HNSW index construction.
|
||||
struct UsearchHnswParams
|
||||
{
|
||||
size_t connectivity = unum::usearch::default_connectivity();
|
||||
size_t expansion_add = unum::usearch::default_expansion_add();
|
||||
size_t connectivity = default_connectivity;
|
||||
size_t expansion_add = default_expansion_add;
|
||||
};
|
||||
|
||||
using USearchIndex = unum::usearch::index_dense_t;
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Core/BaseSettingsProgramOptions.h>
|
||||
#include <Core/MergeSelectorAlgorithm.h>
|
||||
#include <Core/SettingsChangesHistory.h>
|
||||
@ -486,8 +487,7 @@ void MergeTreeColumnSettings::validate(const SettingsChanges & changes)
|
||||
}
|
||||
}
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
|
||||
MergeTreeSettings ## TYPE NAME = & MergeTreeSettings ## Impl :: NAME;
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) MergeTreeSettings##TYPE NAME = &MergeTreeSettingsImpl ::NAME;
|
||||
|
||||
namespace MergeTreeSetting
|
||||
{
|
||||
@ -511,18 +511,7 @@ MergeTreeSettings::MergeTreeSettings(MergeTreeSettings && settings) noexcept
|
||||
|
||||
MergeTreeSettings::~MergeTreeSettings() = default;
|
||||
|
||||
#define IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR(CLASS_NAME, TYPE) \
|
||||
const SettingField##TYPE & MergeTreeSettings::operator[](CLASS_NAME##TYPE t) const \
|
||||
{ \
|
||||
return impl.get()->*t; \
|
||||
} \
|
||||
SettingField##TYPE & MergeTreeSettings::operator[](CLASS_NAME##TYPE t) \
|
||||
{ \
|
||||
return impl.get()->*t; \
|
||||
}
|
||||
|
||||
MERGETREE_SETTINGS_SUPPORTED_TYPES(MergeTreeSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
#undef IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR
|
||||
|
||||
bool MergeTreeSettings::has(std::string_view name) const
|
||||
{
|
||||
|
@ -144,11 +144,7 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context,
|
||||
|
||||
auto client = getClient(url, *s3_settings, context, /* for_disk_s3 */false);
|
||||
auto key_generator = createObjectStorageKeysGeneratorAsIsWithPrefix(url.key);
|
||||
auto s3_capabilities = S3Capabilities
|
||||
{
|
||||
.support_batch_delete = config.getBool("s3.support_batch_delete", true),
|
||||
.support_proxy = config.getBool("s3.support_proxy", config.has("s3.proxy")),
|
||||
};
|
||||
auto s3_capabilities = getCapabilitiesFromConfig(config, "s3");
|
||||
|
||||
return std::make_shared<S3ObjectStorage>(
|
||||
std::move(client), std::move(s3_settings), url, s3_capabilities,
|
||||
|
@ -41,6 +41,11 @@ namespace Setting
|
||||
extern const SettingsBool cloud_mode;
|
||||
}
|
||||
|
||||
namespace ObjectStorageQueueSetting
|
||||
{
|
||||
extern const ObjectStorageQueueSettingsObjectStorageQueueMode mode;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
UInt64 getCurrentTime()
|
||||
@ -227,7 +232,7 @@ ObjectStorageQueueTableMetadata ObjectStorageQueueMetadata::syncWithKeeper(
|
||||
|
||||
std::vector<std::string> metadata_paths;
|
||||
size_t buckets_num = 0;
|
||||
if (settings.mode == ObjectStorageQueueMode::ORDERED)
|
||||
if (settings[ObjectStorageQueueSetting::mode] == ObjectStorageQueueMode::ORDERED)
|
||||
{
|
||||
buckets_num = getBucketsNum(table_metadata);
|
||||
if (buckets_num == 0)
|
||||
|
@ -7,7 +7,6 @@
|
||||
#include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueIFileMetadata.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueOrderedFileMetadata.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
|
||||
@ -17,6 +16,7 @@ namespace Poco { class Logger; }
|
||||
namespace DB
|
||||
{
|
||||
class StorageObjectStorageQueue;
|
||||
struct ObjectStorageQueueSettings;
|
||||
struct ObjectStorageQueueTableMetadata;
|
||||
struct StorageInMemoryMetadata;
|
||||
using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr;
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,8 +1,10 @@
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -13,8 +15,70 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_SETTING;
|
||||
}
|
||||
|
||||
#define OBJECT_STORAGE_QUEUE_RELATED_SETTINGS(M, ALIAS) \
|
||||
M(ObjectStorageQueueMode, mode, ObjectStorageQueueMode::ORDERED, \
|
||||
"With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer." \
|
||||
"With ordered mode, only the max name of the successfully consumed file stored.", \
|
||||
0) \
|
||||
M(ObjectStorageQueueAction, after_processing, ObjectStorageQueueAction::KEEP, "Delete or keep file in after successful processing", 0) \
|
||||
M(String, keeper_path, "", "Zookeeper node path", 0) \
|
||||
M(UInt32, loading_retries, 10, "Retry loading up to specified number of times", 0) \
|
||||
M(UInt32, processing_threads_num, 1, "Number of processing threads", 0) \
|
||||
M(UInt32, enable_logging_to_queue_log, 1, "Enable logging to system table system.(s3/azure_)queue_log", 0) \
|
||||
M(String, last_processed_path, "", "For Ordered mode. Files that have lexicographically smaller file name are considered already processed", 0) \
|
||||
M(UInt32, tracked_file_ttl_sec, 0, "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", 0) \
|
||||
M(UInt32, polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \
|
||||
M(UInt32, polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \
|
||||
M(UInt32, polling_backoff_ms, 1000, "Polling backoff", 0) \
|
||||
M(UInt32, tracked_files_limit, 1000, "For unordered mode. Max set size for tracking processed files in ZooKeeper", 0) \
|
||||
M(UInt32, cleanup_interval_min_ms, 60000, "For unordered mode. Polling backoff min for cleanup", 0) \
|
||||
M(UInt32, cleanup_interval_max_ms, 60000, "For unordered mode. Polling backoff max for cleanup", 0) \
|
||||
M(UInt32, buckets, 0, "Number of buckets for Ordered mode parallel processing", 0) \
|
||||
M(UInt32, max_processed_files_before_commit, 100, "Number of files which can be processed before being committed to keeper", 0) \
|
||||
M(UInt32, max_processed_rows_before_commit, 0, "Number of rows which can be processed before being committed to keeper", 0) \
|
||||
M(UInt32, max_processed_bytes_before_commit, 0, "Number of bytes which can be processed before being committed to keeper", 0) \
|
||||
M(UInt32, max_processing_time_sec_before_commit, 0, "Timeout in seconds after which to commit files committed to keeper", 0) \
|
||||
|
||||
#define LIST_OF_OBJECT_STORAGE_QUEUE_SETTINGS(M, ALIAS) \
|
||||
OBJECT_STORAGE_QUEUE_RELATED_SETTINGS(M, ALIAS) \
|
||||
LIST_OF_ALL_FORMAT_SETTINGS(M, ALIAS)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(ObjectStorageQueueSettingsTraits, LIST_OF_OBJECT_STORAGE_QUEUE_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(ObjectStorageQueueSettingsTraits, LIST_OF_OBJECT_STORAGE_QUEUE_SETTINGS)
|
||||
|
||||
struct ObjectStorageQueueSettingsImpl : public BaseSettings<ObjectStorageQueueSettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
|
||||
ObjectStorageQueueSettings##TYPE NAME = &ObjectStorageQueueSettingsImpl ::NAME;
|
||||
|
||||
namespace ObjectStorageQueueSetting
|
||||
{
|
||||
LIST_OF_OBJECT_STORAGE_QUEUE_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
}
|
||||
|
||||
#undef INITIALIZE_SETTING_EXTERN
|
||||
|
||||
ObjectStorageQueueSettings::ObjectStorageQueueSettings() : impl(std::make_unique<ObjectStorageQueueSettingsImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
ObjectStorageQueueSettings::ObjectStorageQueueSettings(const ObjectStorageQueueSettings & settings)
|
||||
: impl(std::make_unique<ObjectStorageQueueSettingsImpl>(*settings.impl))
|
||||
{
|
||||
}
|
||||
|
||||
ObjectStorageQueueSettings::ObjectStorageQueueSettings(ObjectStorageQueueSettings && settings) noexcept
|
||||
: impl(std::make_unique<ObjectStorageQueueSettingsImpl>(std::move(*settings.impl)))
|
||||
{
|
||||
}
|
||||
|
||||
ObjectStorageQueueSettings::~ObjectStorageQueueSettings() = default;
|
||||
|
||||
OBJECT_STORAGE_QUEUE_SETTINGS_SUPPORTED_TYPES(ObjectStorageQueueSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
|
||||
void ObjectStorageQueueSettings::loadFromQuery(ASTStorage & storage_def)
|
||||
{
|
||||
if (storage_def.settings)
|
||||
@ -42,7 +106,7 @@ void ObjectStorageQueueSettings::loadFromQuery(ASTStorage & storage_def)
|
||||
for (const auto & setting : ignore_settings)
|
||||
settings_changes.removeSetting(setting);
|
||||
|
||||
applyChanges(settings_changes);
|
||||
impl->applyChanges(settings_changes);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
|
@ -1,51 +1,59 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTStorage;
|
||||
struct ObjectStorageQueueSettingsImpl;
|
||||
|
||||
/// List of available types supported in ObjectStorageQueueSettings object
|
||||
#define OBJECT_STORAGE_QUEUE_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, ArrowCompression) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, CapnProtoEnumComparingMode) \
|
||||
M(CLASS_NAME, Char) \
|
||||
M(CLASS_NAME, DateTimeInputFormat) \
|
||||
M(CLASS_NAME, DateTimeOutputFormat) \
|
||||
M(CLASS_NAME, DateTimeOverflowBehavior) \
|
||||
M(CLASS_NAME, Double) \
|
||||
M(CLASS_NAME, EscapingRule) \
|
||||
M(CLASS_NAME, Float) \
|
||||
M(CLASS_NAME, IdentifierQuotingRule) \
|
||||
M(CLASS_NAME, IdentifierQuotingStyle) \
|
||||
M(CLASS_NAME, Int64) \
|
||||
M(CLASS_NAME, IntervalOutputFormat) \
|
||||
M(CLASS_NAME, MsgPackUUIDRepresentation) \
|
||||
M(CLASS_NAME, ObjectStorageQueueAction) \
|
||||
M(CLASS_NAME, ObjectStorageQueueMode) \
|
||||
M(CLASS_NAME, ORCCompression) \
|
||||
M(CLASS_NAME, ParquetCompression) \
|
||||
M(CLASS_NAME, ParquetVersion) \
|
||||
M(CLASS_NAME, SchemaInferenceMode) \
|
||||
M(CLASS_NAME, String) \
|
||||
M(CLASS_NAME, UInt32) \
|
||||
M(CLASS_NAME, UInt64) \
|
||||
M(CLASS_NAME, UInt64Auto) \
|
||||
M(CLASS_NAME, URI)
|
||||
|
||||
#define OBJECT_STORAGE_QUEUE_RELATED_SETTINGS(M, ALIAS) \
|
||||
M(ObjectStorageQueueMode, \
|
||||
mode, \
|
||||
ObjectStorageQueueMode::ORDERED, \
|
||||
"With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer." \
|
||||
"With ordered mode, only the max name of the successfully consumed file stored.", \
|
||||
0) \
|
||||
M(ObjectStorageQueueAction, after_processing, ObjectStorageQueueAction::KEEP, "Delete or keep file in after successful processing", 0) \
|
||||
M(String, keeper_path, "", "Zookeeper node path", 0) \
|
||||
M(UInt32, loading_retries, 10, "Retry loading up to specified number of times", 0) \
|
||||
M(UInt32, processing_threads_num, 1, "Number of processing threads", 0) \
|
||||
M(UInt32, enable_logging_to_queue_log, 1, "Enable logging to system table system.(s3/azure_)queue_log", 0) \
|
||||
M(String, last_processed_path, "", "For Ordered mode. Files that have lexicographically smaller file name are considered already processed", 0) \
|
||||
M(UInt32, tracked_file_ttl_sec, 0, "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", 0) \
|
||||
M(UInt32, polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \
|
||||
M(UInt32, polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \
|
||||
M(UInt32, polling_backoff_ms, 1000, "Polling backoff", 0) \
|
||||
M(UInt32, tracked_files_limit, 1000, "For unordered mode. Max set size for tracking processed files in ZooKeeper", 0) \
|
||||
M(UInt32, cleanup_interval_min_ms, 60000, "For unordered mode. Polling backoff min for cleanup", 0) \
|
||||
M(UInt32, cleanup_interval_max_ms, 60000, "For unordered mode. Polling backoff max for cleanup", 0) \
|
||||
M(UInt32, buckets, 0, "Number of buckets for Ordered mode parallel processing", 0) \
|
||||
M(UInt32, max_processed_files_before_commit, 100, "Number of files which can be processed before being committed to keeper", 0) \
|
||||
M(UInt32, max_processed_rows_before_commit, 0, "Number of rows which can be processed before being committed to keeper", 0) \
|
||||
M(UInt32, max_processed_bytes_before_commit, 0, "Number of bytes which can be processed before being committed to keeper", 0) \
|
||||
M(UInt32, max_processing_time_sec_before_commit, 0, "Timeout in seconds after which to commit files committed to keeper", 0) \
|
||||
OBJECT_STORAGE_QUEUE_SETTINGS_SUPPORTED_TYPES(ObjectStorageQueueSettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
#define LIST_OF_OBJECT_STORAGE_QUEUE_SETTINGS(M, ALIAS) \
|
||||
OBJECT_STORAGE_QUEUE_RELATED_SETTINGS(M, ALIAS) \
|
||||
LIST_OF_ALL_FORMAT_SETTINGS(M, ALIAS)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(ObjectStorageQueueSettingsTraits, LIST_OF_OBJECT_STORAGE_QUEUE_SETTINGS)
|
||||
|
||||
|
||||
struct ObjectStorageQueueSettings : public BaseSettings<ObjectStorageQueueSettingsTraits>
|
||||
struct ObjectStorageQueueSettings
|
||||
{
|
||||
void loadFromQuery(ASTStorage & storage_def);
|
||||
};
|
||||
ObjectStorageQueueSettings();
|
||||
ObjectStorageQueueSettings(const ObjectStorageQueueSettings & settings);
|
||||
ObjectStorageQueueSettings(ObjectStorageQueueSettings && settings) noexcept;
|
||||
~ObjectStorageQueueSettings();
|
||||
|
||||
OBJECT_STORAGE_QUEUE_SETTINGS_SUPPORTED_TYPES(ObjectStorageQueueSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void loadFromQuery(ASTStorage & storage_def);
|
||||
|
||||
private:
|
||||
std::unique_ptr<ObjectStorageQueueSettingsImpl> impl;
|
||||
};
|
||||
}
|
||||
|
@ -23,6 +23,15 @@ namespace Setting
|
||||
extern const SettingsMaxThreads max_parsing_threads;
|
||||
}
|
||||
|
||||
namespace ObjectStorageQueueSetting
|
||||
{
|
||||
extern const ObjectStorageQueueSettingsObjectStorageQueueAction after_processing;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processed_bytes_before_commit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processed_files_before_commit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processed_rows_before_commit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processing_time_sec_before_commit;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
|
@ -1,12 +1,13 @@
|
||||
#pragma once
|
||||
#include "config.h"
|
||||
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <Interpreters/ObjectStorageQueueLog.h>
|
||||
#include <Processors/ISource.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h>
|
||||
#include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
#include <Storages/ObjectStorage/StorageObjectStorageSource.h>
|
||||
#include <Interpreters/ObjectStorageQueueLog.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
|
@ -12,6 +12,19 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ObjectStorageQueueSetting
|
||||
{
|
||||
extern const ObjectStorageQueueSettingsObjectStorageQueueAction after_processing;
|
||||
extern const ObjectStorageQueueSettingsUInt32 buckets;
|
||||
extern const ObjectStorageQueueSettingsString last_processed_path;
|
||||
extern const ObjectStorageQueueSettingsUInt32 loading_retries;
|
||||
extern const ObjectStorageQueueSettingsObjectStorageQueueMode mode;
|
||||
extern const ObjectStorageQueueSettingsUInt32 processing_threads_num;
|
||||
extern const ObjectStorageQueueSettingsUInt32 tracked_files_limit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 tracked_file_ttl_sec;
|
||||
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int METADATA_MISMATCH;
|
||||
@ -43,19 +56,19 @@ ObjectStorageQueueTableMetadata::ObjectStorageQueueTableMetadata(
|
||||
const std::string & format_)
|
||||
: format_name(format_)
|
||||
, columns(columns_.toString())
|
||||
, after_processing(engine_settings.after_processing.toString())
|
||||
, mode(engine_settings.mode.toString())
|
||||
, tracked_files_limit(engine_settings.tracked_files_limit)
|
||||
, tracked_files_ttl_sec(engine_settings.tracked_file_ttl_sec)
|
||||
, buckets(engine_settings.buckets)
|
||||
, last_processed_path(engine_settings.last_processed_path)
|
||||
, loading_retries(engine_settings.loading_retries)
|
||||
, after_processing(engine_settings[ObjectStorageQueueSetting::after_processing].toString())
|
||||
, mode(engine_settings[ObjectStorageQueueSetting::mode].toString())
|
||||
, tracked_files_limit(engine_settings[ObjectStorageQueueSetting::tracked_files_limit])
|
||||
, tracked_files_ttl_sec(engine_settings[ObjectStorageQueueSetting::tracked_file_ttl_sec])
|
||||
, buckets(engine_settings[ObjectStorageQueueSetting::buckets])
|
||||
, last_processed_path(engine_settings[ObjectStorageQueueSetting::last_processed_path])
|
||||
, loading_retries(engine_settings[ObjectStorageQueueSetting::loading_retries])
|
||||
{
|
||||
processing_threads_num_changed = engine_settings.processing_threads_num.changed;
|
||||
if (!processing_threads_num_changed && engine_settings.processing_threads_num <= 1)
|
||||
processing_threads_num_changed = engine_settings[ObjectStorageQueueSetting::processing_threads_num].changed;
|
||||
if (!processing_threads_num_changed && engine_settings[ObjectStorageQueueSetting::processing_threads_num] <= 1)
|
||||
processing_threads_num = std::max<uint32_t>(getNumberOfCPUCoresToUse(), 16);
|
||||
else
|
||||
processing_threads_num = engine_settings.processing_threads_num;
|
||||
processing_threads_num = engine_settings[ObjectStorageQueueSetting::processing_threads_num];
|
||||
}
|
||||
|
||||
String ObjectStorageQueueTableMetadata::toString() const
|
||||
|
@ -1,15 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <base/types.h>
|
||||
#include <Poco/JSON/JSON.h>
|
||||
#include <Poco/JSON/Object.h>
|
||||
#include <base/types.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct ObjectStorageQueueSettings;
|
||||
class WriteBuffer;
|
||||
class ReadBuffer;
|
||||
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <Storages/ObjectStorageQueue/StorageObjectStorageQueue.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueMetadataFactory.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Storages/StorageMaterializedView.h>
|
||||
#include <Storages/StorageSnapshot.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
@ -39,6 +40,23 @@ namespace Setting
|
||||
extern const SettingsBool use_concurrency_control;
|
||||
}
|
||||
|
||||
namespace ObjectStorageQueueSetting
|
||||
{
|
||||
extern const ObjectStorageQueueSettingsUInt32 cleanup_interval_max_ms;
|
||||
extern const ObjectStorageQueueSettingsUInt32 cleanup_interval_min_ms;
|
||||
extern const ObjectStorageQueueSettingsUInt32 enable_logging_to_queue_log;
|
||||
extern const ObjectStorageQueueSettingsString keeper_path;
|
||||
extern const ObjectStorageQueueSettingsObjectStorageQueueMode mode;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processed_bytes_before_commit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processed_files_before_commit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processed_rows_before_commit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 max_processing_time_sec_before_commit;
|
||||
extern const ObjectStorageQueueSettingsUInt32 polling_min_timeout_ms;
|
||||
extern const ObjectStorageQueueSettingsUInt32 polling_max_timeout_ms;
|
||||
extern const ObjectStorageQueueSettingsUInt32 polling_backoff_ms;
|
||||
extern const ObjectStorageQueueSettingsUInt32 processing_threads_num;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -56,10 +74,10 @@ namespace
|
||||
zk_path_prefix = "/";
|
||||
|
||||
std::string result_zk_path;
|
||||
if (queue_settings.keeper_path.changed)
|
||||
if (queue_settings[ObjectStorageQueueSetting::keeper_path].changed)
|
||||
{
|
||||
/// We do not add table uuid here on purpose.
|
||||
result_zk_path = fs::path(zk_path_prefix) / queue_settings.keeper_path.value;
|
||||
result_zk_path = fs::path(zk_path_prefix) / queue_settings[ObjectStorageQueueSetting::keeper_path].value;
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -73,22 +91,22 @@ namespace
|
||||
ObjectStorageQueueSettings & queue_settings,
|
||||
bool is_attach)
|
||||
{
|
||||
if (!is_attach && !queue_settings.mode.changed)
|
||||
if (!is_attach && !queue_settings[ObjectStorageQueueSetting::mode].changed)
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `mode` (Unordered/Ordered) is not specified, but is required.");
|
||||
}
|
||||
/// In case !is_attach, we leave Ordered mode as default for compatibility.
|
||||
|
||||
if (!queue_settings.processing_threads_num)
|
||||
if (!queue_settings[ObjectStorageQueueSetting::processing_threads_num])
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `processing_threads_num` cannot be set to zero");
|
||||
}
|
||||
|
||||
if (queue_settings.cleanup_interval_min_ms > queue_settings.cleanup_interval_max_ms)
|
||||
if (queue_settings[ObjectStorageQueueSetting::cleanup_interval_min_ms] > queue_settings[ObjectStorageQueueSetting::cleanup_interval_max_ms])
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Setting `cleanup_interval_min_ms` ({}) must be less or equal to `cleanup_interval_max_ms` ({})",
|
||||
queue_settings.cleanup_interval_min_ms, queue_settings.cleanup_interval_max_ms);
|
||||
queue_settings[ObjectStorageQueueSetting::cleanup_interval_min_ms], queue_settings[ObjectStorageQueueSetting::cleanup_interval_max_ms]);
|
||||
}
|
||||
}
|
||||
|
||||
@ -132,19 +150,19 @@ StorageObjectStorageQueue::StorageObjectStorageQueue(
|
||||
: IStorage(table_id_)
|
||||
, WithContext(context_)
|
||||
, zk_path(chooseZooKeeperPath(table_id_, context_->getSettingsRef(), *queue_settings_))
|
||||
, enable_logging_to_queue_log(queue_settings_->enable_logging_to_queue_log)
|
||||
, polling_min_timeout_ms(queue_settings_->polling_min_timeout_ms)
|
||||
, polling_max_timeout_ms(queue_settings_->polling_max_timeout_ms)
|
||||
, polling_backoff_ms(queue_settings_->polling_backoff_ms)
|
||||
, enable_logging_to_queue_log((*queue_settings_)[ObjectStorageQueueSetting::enable_logging_to_queue_log])
|
||||
, polling_min_timeout_ms((*queue_settings_)[ObjectStorageQueueSetting::polling_min_timeout_ms])
|
||||
, polling_max_timeout_ms((*queue_settings_)[ObjectStorageQueueSetting::polling_max_timeout_ms])
|
||||
, polling_backoff_ms((*queue_settings_)[ObjectStorageQueueSetting::polling_backoff_ms])
|
||||
, commit_settings(CommitSettings{
|
||||
.max_processed_files_before_commit = queue_settings_->max_processed_files_before_commit,
|
||||
.max_processed_rows_before_commit = queue_settings_->max_processed_rows_before_commit,
|
||||
.max_processed_bytes_before_commit = queue_settings_->max_processed_bytes_before_commit,
|
||||
.max_processing_time_sec_before_commit = queue_settings_->max_processing_time_sec_before_commit,
|
||||
.max_processed_files_before_commit = (*queue_settings_)[ObjectStorageQueueSetting::max_processed_files_before_commit],
|
||||
.max_processed_rows_before_commit = (*queue_settings_)[ObjectStorageQueueSetting::max_processed_rows_before_commit],
|
||||
.max_processed_bytes_before_commit = (*queue_settings_)[ObjectStorageQueueSetting::max_processed_bytes_before_commit],
|
||||
.max_processing_time_sec_before_commit = (*queue_settings_)[ObjectStorageQueueSetting::max_processing_time_sec_before_commit],
|
||||
})
|
||||
, configuration{configuration_}
|
||||
, format_settings(format_settings_)
|
||||
, reschedule_processing_interval_ms(queue_settings_->polling_min_timeout_ms)
|
||||
, reschedule_processing_interval_ms((*queue_settings_)[ObjectStorageQueueSetting::polling_min_timeout_ms])
|
||||
, log(getLogger(fmt::format("Storage{}Queue ({})", configuration->getEngineName(), table_id_.getFullTableName())))
|
||||
{
|
||||
if (configuration->getPath().empty())
|
||||
@ -185,7 +203,7 @@ StorageObjectStorageQueue::StorageObjectStorageQueue(
|
||||
zk_path, *queue_settings_, storage_metadata.getColumns(), configuration_->format, context_, is_attach, log);
|
||||
|
||||
auto queue_metadata = std::make_unique<ObjectStorageQueueMetadata>(
|
||||
zk_path, std::move(table_metadata), queue_settings_->cleanup_interval_min_ms, queue_settings_->cleanup_interval_max_ms);
|
||||
zk_path, std::move(table_metadata), (*queue_settings_)[ObjectStorageQueueSetting::cleanup_interval_min_ms], (*queue_settings_)[ObjectStorageQueueSetting::cleanup_interval_max_ms]);
|
||||
|
||||
files_metadata = ObjectStorageQueueMetadataFactory::instance().getOrCreate(zk_path, std::move(queue_metadata));
|
||||
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h>
|
||||
#include <Storages/ObjectStorageQueue/ObjectStorageQueueSource.h>
|
||||
#include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
#include <Interpreters/Context.h>
|
||||
@ -15,6 +14,7 @@
|
||||
namespace DB
|
||||
{
|
||||
class ObjectStorageQueueMetadata;
|
||||
struct ObjectStorageQueueSettings;
|
||||
|
||||
class StorageObjectStorageQueue : public IStorage, WithContext
|
||||
{
|
||||
|
@ -1,6 +1,9 @@
|
||||
#include "MaterializedPostgreSQLSettings.h"
|
||||
#include <Storages/PostgreSQL/MaterializedPostgreSQLSettings.h>
|
||||
|
||||
#if USE_LIBPQXX
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -15,15 +18,72 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_SETTING;
|
||||
}
|
||||
|
||||
#define LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS(M, ALIAS) \
|
||||
M(UInt64, materialized_postgresql_max_block_size, 65536, "Number of row collected before flushing data into table.", 0) \
|
||||
M(String, materialized_postgresql_tables_list, "", "List of tables for MaterializedPostgreSQL database engine", 0) \
|
||||
M(String, materialized_postgresql_schema_list, "", "List of schemas for MaterializedPostgreSQL database engine", 0) \
|
||||
M(String, materialized_postgresql_replication_slot, "", "A user-created replication slot", 0) \
|
||||
M(String, materialized_postgresql_snapshot, "", "User provided snapshot in case he manages replication slots himself", 0) \
|
||||
M(String, materialized_postgresql_schema, "", "PostgreSQL schema", 0) \
|
||||
M(Bool, materialized_postgresql_tables_list_with_schema, false, \
|
||||
"Consider by default that if there is a dot in tables list 'name.name', " \
|
||||
"then the first name is postgres schema and second is postgres table. This setting is needed to allow table names with dots", 0) \
|
||||
M(UInt64, materialized_postgresql_backoff_min_ms, 200, "Poll backoff start point", 0) \
|
||||
M(UInt64, materialized_postgresql_backoff_max_ms, 10000, "Poll backoff max point", 0) \
|
||||
M(UInt64, materialized_postgresql_backoff_factor, 2, "Poll backoff factor", 0) \
|
||||
M(Bool, materialized_postgresql_use_unique_replication_consumer_identifier, false, "Should a unique consumer be registered for table replication", 0) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS)
|
||||
|
||||
struct MaterializedPostgreSQLSettingsImpl : public BaseSettings<MaterializedPostgreSQLSettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) MaterializedPostgreSQLSettings##TYPE NAME = &MaterializedPostgreSQLSettingsImpl ::NAME;
|
||||
|
||||
namespace MaterializedPostgreSQLSetting
|
||||
{
|
||||
LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
}
|
||||
|
||||
#undef INITIALIZE_SETTING_EXTERN
|
||||
|
||||
MaterializedPostgreSQLSettings::MaterializedPostgreSQLSettings() : impl(std::make_unique<MaterializedPostgreSQLSettingsImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
MaterializedPostgreSQLSettings::MaterializedPostgreSQLSettings(const MaterializedPostgreSQLSettings & settings)
|
||||
: impl(std::make_unique<MaterializedPostgreSQLSettingsImpl>(*settings.impl))
|
||||
{
|
||||
}
|
||||
|
||||
MaterializedPostgreSQLSettings::MaterializedPostgreSQLSettings(MaterializedPostgreSQLSettings && settings) noexcept
|
||||
: impl(std::make_unique<MaterializedPostgreSQLSettingsImpl>(std::move(*settings.impl)))
|
||||
{
|
||||
}
|
||||
|
||||
MaterializedPostgreSQLSettings::~MaterializedPostgreSQLSettings() = default;
|
||||
|
||||
MATERIALIZED_POSTGRESQL_SETTINGS_SUPPORTED_TYPES(MaterializedPostgreSQLSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void MaterializedPostgreSQLSettings::applyChange(const SettingChange & change)
|
||||
{
|
||||
impl->applyChange(change);
|
||||
}
|
||||
|
||||
bool MaterializedPostgreSQLSettings::has(std::string_view name) const
|
||||
{
|
||||
return impl->has(name);
|
||||
}
|
||||
|
||||
void MaterializedPostgreSQLSettings::loadFromQuery(ASTStorage & storage_def)
|
||||
{
|
||||
if (storage_def.settings)
|
||||
{
|
||||
try
|
||||
{
|
||||
applyChanges(storage_def.settings->changes);
|
||||
impl->applyChanges(storage_def.settings->changes);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
|
@ -3,34 +3,40 @@
|
||||
#include "config.h"
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Core/BaseSettings.h>
|
||||
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTStorage;
|
||||
class ASTStorage;
|
||||
struct SettingChange;
|
||||
struct MaterializedPostgreSQLSettingsImpl;
|
||||
|
||||
#define MATERIALIZED_POSTGRESQL_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, String) \
|
||||
M(CLASS_NAME, UInt64)
|
||||
|
||||
MATERIALIZED_POSTGRESQL_SETTINGS_SUPPORTED_TYPES(MaterializedPostgreSQLSettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
|
||||
#define LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS(M, ALIAS) \
|
||||
M(UInt64, materialized_postgresql_max_block_size, 65536, "Number of row collected before flushing data into table.", 0) \
|
||||
M(String, materialized_postgresql_tables_list, "", "List of tables for MaterializedPostgreSQL database engine", 0) \
|
||||
M(String, materialized_postgresql_schema_list, "", "List of schemas for MaterializedPostgreSQL database engine", 0) \
|
||||
M(String, materialized_postgresql_replication_slot, "", "A user-created replication slot", 0) \
|
||||
M(String, materialized_postgresql_snapshot, "", "User provided snapshot in case he manages replication slots himself", 0) \
|
||||
M(String, materialized_postgresql_schema, "", "PostgreSQL schema", 0) \
|
||||
M(Bool, materialized_postgresql_tables_list_with_schema, false, \
|
||||
"Consider by default that if there is a dot in tables list 'name.name', " \
|
||||
"then the first name is postgres schema and second is postgres table. This setting is needed to allow table names with dots", 0) \
|
||||
M(UInt64, materialized_postgresql_backoff_min_ms, 200, "Poll backoff start point", 0) \
|
||||
M(UInt64, materialized_postgresql_backoff_max_ms, 10000, "Poll backoff max point", 0) \
|
||||
M(UInt64, materialized_postgresql_backoff_factor, 2, "Poll backoff factor", 0) \
|
||||
M(Bool, materialized_postgresql_use_unique_replication_consumer_identifier, false, "Should a unique consumer be registered for table replication", 0) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS)
|
||||
|
||||
struct MaterializedPostgreSQLSettings : public BaseSettings<MaterializedPostgreSQLSettingsTraits>
|
||||
struct MaterializedPostgreSQLSettings
|
||||
{
|
||||
MaterializedPostgreSQLSettings();
|
||||
MaterializedPostgreSQLSettings(const MaterializedPostgreSQLSettings & settings);
|
||||
MaterializedPostgreSQLSettings(MaterializedPostgreSQLSettings && settings) noexcept;
|
||||
~MaterializedPostgreSQLSettings();
|
||||
|
||||
MATERIALIZED_POSTGRESQL_SETTINGS_SUPPORTED_TYPES(MaterializedPostgreSQLSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void applyChange(const SettingChange & change);
|
||||
bool has(std::string_view name) const;
|
||||
void loadFromQuery(ASTStorage & storage_def);
|
||||
|
||||
private:
|
||||
std::unique_ptr<MaterializedPostgreSQLSettingsImpl> impl;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include "PostgreSQLReplicationHandler.h"
|
||||
|
||||
#include <base/sort.h>
|
||||
|
||||
@ -9,6 +8,8 @@
|
||||
#include <QueryPipeline/QueryPipeline.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
|
||||
#include <Storages/PostgreSQL/MaterializedPostgreSQLSettings.h>
|
||||
#include <Storages/PostgreSQL/PostgreSQLReplicationHandler.h>
|
||||
#include <Storages/PostgreSQL/StorageMaterializedPostgreSQL.h>
|
||||
#include <Interpreters/getTableOverride.h>
|
||||
#include <Interpreters/InterpreterDropQuery.h>
|
||||
@ -26,6 +27,21 @@ namespace DB
|
||||
static const auto CLEANUP_RESCHEDULE_MS = 600000 * 3; /// 30 min
|
||||
static constexpr size_t replication_slot_name_max_size = 64;
|
||||
|
||||
namespace MaterializedPostgreSQLSetting
|
||||
{
|
||||
extern const MaterializedPostgreSQLSettingsUInt64 materialized_postgresql_backoff_factor;
|
||||
extern const MaterializedPostgreSQLSettingsUInt64 materialized_postgresql_backoff_max_ms;
|
||||
extern const MaterializedPostgreSQLSettingsUInt64 materialized_postgresql_backoff_min_ms;
|
||||
extern const MaterializedPostgreSQLSettingsUInt64 materialized_postgresql_max_block_size;
|
||||
extern const MaterializedPostgreSQLSettingsString materialized_postgresql_replication_slot;
|
||||
extern const MaterializedPostgreSQLSettingsString materialized_postgresql_schema;
|
||||
extern const MaterializedPostgreSQLSettingsString materialized_postgresql_schema_list;
|
||||
extern const MaterializedPostgreSQLSettingsString materialized_postgresql_snapshot;
|
||||
extern const MaterializedPostgreSQLSettingsString materialized_postgresql_tables_list;
|
||||
extern const MaterializedPostgreSQLSettingsBool materialized_postgresql_tables_list_with_schema;
|
||||
extern const MaterializedPostgreSQLSettingsBool materialized_postgresql_use_unique_replication_consumer_identifier;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -103,10 +119,10 @@ namespace
|
||||
const String & clickhouse_uuid,
|
||||
const MaterializedPostgreSQLSettings & replication_settings)
|
||||
{
|
||||
String slot_name = replication_settings.materialized_postgresql_replication_slot;
|
||||
String slot_name = replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_replication_slot];
|
||||
if (slot_name.empty())
|
||||
{
|
||||
if (replication_settings.materialized_postgresql_use_unique_replication_consumer_identifier)
|
||||
if (replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_use_unique_replication_consumer_identifier])
|
||||
slot_name = clickhouse_uuid;
|
||||
else
|
||||
slot_name = postgres_table.empty() ? postgres_database : fmt::format("{}_{}_ch_replication_slot", postgres_database, postgres_table);
|
||||
@ -131,22 +147,22 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler(
|
||||
, log(getLogger("PostgreSQLReplicationHandler"))
|
||||
, is_attach(is_attach_)
|
||||
, postgres_database(postgres_database_)
|
||||
, postgres_schema(replication_settings.materialized_postgresql_schema)
|
||||
, postgres_schema(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_schema])
|
||||
, current_database_name(clickhouse_database_)
|
||||
, connection_info(connection_info_)
|
||||
, max_block_size(replication_settings.materialized_postgresql_max_block_size)
|
||||
, max_block_size(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_max_block_size])
|
||||
, is_materialized_postgresql_database(is_materialized_postgresql_database_)
|
||||
, tables_list(replication_settings.materialized_postgresql_tables_list)
|
||||
, schema_list(replication_settings.materialized_postgresql_schema_list)
|
||||
, schema_as_a_part_of_table_name(!schema_list.empty() || replication_settings.materialized_postgresql_tables_list_with_schema)
|
||||
, user_managed_slot(!replication_settings.materialized_postgresql_replication_slot.value.empty())
|
||||
, user_provided_snapshot(replication_settings.materialized_postgresql_snapshot)
|
||||
, tables_list(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_tables_list])
|
||||
, schema_list(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_schema_list])
|
||||
, schema_as_a_part_of_table_name(!schema_list.empty() || replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_tables_list_with_schema])
|
||||
, user_managed_slot(!replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_replication_slot].value.empty())
|
||||
, user_provided_snapshot(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_snapshot])
|
||||
, replication_slot(getReplicationSlotName(postgres_database_, postgres_table_, clickhouse_uuid_, replication_settings))
|
||||
, tmp_replication_slot(replication_slot + "_tmp")
|
||||
, publication_name(getPublicationName(postgres_database_, postgres_table_))
|
||||
, reschedule_backoff_min_ms(replication_settings.materialized_postgresql_backoff_min_ms)
|
||||
, reschedule_backoff_max_ms(replication_settings.materialized_postgresql_backoff_max_ms)
|
||||
, reschedule_backoff_factor(replication_settings.materialized_postgresql_backoff_factor)
|
||||
, reschedule_backoff_min_ms(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_backoff_min_ms])
|
||||
, reschedule_backoff_max_ms(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_backoff_max_ms])
|
||||
, reschedule_backoff_factor(replication_settings[MaterializedPostgreSQLSetting::materialized_postgresql_backoff_factor])
|
||||
, milliseconds_to_wait(reschedule_backoff_min_ms)
|
||||
{
|
||||
if (!schema_list.empty() && !tables_list.empty())
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include "MaterializedPostgreSQLConsumer.h"
|
||||
#include "MaterializedPostgreSQLSettings.h"
|
||||
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
|
||||
#include <Core/PostgreSQL/Utils.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
@ -10,6 +9,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct MaterializedPostgreSQLSettings;
|
||||
class StorageMaterializedPostgreSQL;
|
||||
struct SettingChange;
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include "StorageMaterializedPostgreSQL.h"
|
||||
#include <Storages/PostgreSQL/StorageMaterializedPostgreSQL.h>
|
||||
#include <Storages/PostgreSQL/MaterializedPostgreSQLSettings.h>
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Common/logger_useful.h>
|
||||
@ -46,6 +47,11 @@ namespace Setting
|
||||
extern const SettingsUInt64 postgresql_connection_attempt_timeout;
|
||||
}
|
||||
|
||||
namespace MaterializedPostgreSQLSetting
|
||||
{
|
||||
extern const MaterializedPostgreSQLSettingsString materialized_postgresql_tables_list;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -79,7 +85,7 @@ StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL(
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
setVirtuals(createVirtuals());
|
||||
|
||||
replication_settings->materialized_postgresql_tables_list = remote_table_name_;
|
||||
(*replication_settings)[MaterializedPostgreSQLSetting::materialized_postgresql_tables_list] = remote_table_name_;
|
||||
|
||||
replication_handler = std::make_unique<PostgreSQLReplicationHandler>(
|
||||
remote_database_name,
|
||||
|
@ -4,7 +4,6 @@
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include "PostgreSQLReplicationHandler.h"
|
||||
#include "MaterializedPostgreSQLSettings.h"
|
||||
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
@ -17,6 +16,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct MaterializedPostgreSQLSettings;
|
||||
|
||||
/** TODO list:
|
||||
* - Actually I think we can support ddl even though logical replication does not fully support it.
|
||||
|
@ -1,8 +1,12 @@
|
||||
#include <Storages/RabbitMQ/RabbitMQSettings.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Storages/RabbitMQ/RabbitMQSettings.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/NamedCollections/NamedCollections.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -12,15 +16,84 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_SETTING;
|
||||
}
|
||||
|
||||
#define RABBITMQ_RELATED_SETTINGS(M, ALIAS) \
|
||||
M(String, rabbitmq_host_port, "", "A host-port to connect to RabbitMQ server.", 0) \
|
||||
M(String, rabbitmq_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \
|
||||
M(String, rabbitmq_format, "", "The message format.", 0) \
|
||||
M(String, rabbitmq_exchange_type, "default", "The exchange type.", 0) \
|
||||
M(String, rabbitmq_routing_key_list, "5672", "A string of routing keys, separated by dots.", 0) \
|
||||
M(String, rabbitmq_schema, "", "Schema identifier (used by schema-based formats) for RabbitMQ engine", 0) \
|
||||
M(UInt64, rabbitmq_num_consumers, 1, "The number of consumer channels per table.", 0) \
|
||||
M(UInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \
|
||||
M(String, rabbitmq_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \
|
||||
M(Bool, rabbitmq_persistent, false, "For insert query messages will be made 'persistent', durable.", 0) \
|
||||
M(Bool, rabbitmq_secure, false, "Use SSL connection", 0) \
|
||||
M(String, rabbitmq_address, "", "Address for connection", 0) \
|
||||
M(UInt64, rabbitmq_skip_broken_messages, 0, "Skip at least this number of broken messages from RabbitMQ per block", 0) \
|
||||
M(UInt64, rabbitmq_max_block_size, 0, "Number of row collected before flushing data from RabbitMQ.", 0) \
|
||||
M(UInt64, rabbitmq_flush_interval_ms, 0, "Timeout for flushing data from RabbitMQ.", 0) \
|
||||
M(String, rabbitmq_vhost, "/", "RabbitMQ vhost.", 0) \
|
||||
M(String, rabbitmq_queue_settings_list, "", "A list of rabbitmq queue settings", 0) \
|
||||
M(UInt64, rabbitmq_empty_queue_backoff_start_ms, 10, "A minimum backoff point to reschedule read if the rabbitmq queue is empty", 0) \
|
||||
M(UInt64, rabbitmq_empty_queue_backoff_end_ms, 10000, "A maximum backoff point to reschedule read if the rabbitmq queue is empty", 0) \
|
||||
M(UInt64, rabbitmq_empty_queue_backoff_step_ms, 100, "A backoff step to reschedule read if the rabbitmq queue is empty", 0) \
|
||||
M(Bool, rabbitmq_queue_consume, false, "Use user-defined queues and do not make any RabbitMQ setup: declaring exchanges, queues, bindings", 0) \
|
||||
M(String, rabbitmq_username, "", "RabbitMQ username", 0) \
|
||||
M(String, rabbitmq_password, "", "RabbitMQ password", 0) \
|
||||
M(Bool, reject_unhandled_messages, false, "Allow messages to be rejected in case they cannot be processed. This also automatically implies if there is a x-deadletter-exchange queue setting added", 0) \
|
||||
M(Bool, rabbitmq_commit_on_select, false, "Commit messages when select query is made", 0) \
|
||||
M(UInt64, rabbitmq_max_rows_per_message, 1, "The maximum number of rows produced in one message for row-based formats.", 0) \
|
||||
M(StreamingHandleErrorMode, rabbitmq_handle_error_mode, StreamingHandleErrorMode::DEFAULT, "How to handle errors for RabbitMQ engine. Possible values: default (throw an exception after rabbitmq_skip_broken_messages broken messages), stream (save broken messages and errors in virtual columns _raw_message, _error).", 0) \
|
||||
|
||||
#define OBSOLETE_RABBITMQ_SETTINGS(M, ALIAS) \
|
||||
MAKE_OBSOLETE(M, Char, rabbitmq_row_delimiter, '\0') \
|
||||
|
||||
#define LIST_OF_RABBITMQ_SETTINGS(M, ALIAS) \
|
||||
RABBITMQ_RELATED_SETTINGS(M, ALIAS) \
|
||||
OBSOLETE_RABBITMQ_SETTINGS(M, ALIAS) \
|
||||
LIST_OF_ALL_FORMAT_SETTINGS(M, ALIAS) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(RabbitMQSettingsTraits, LIST_OF_RABBITMQ_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(RabbitMQSettingsTraits, LIST_OF_RABBITMQ_SETTINGS)
|
||||
|
||||
struct RabbitMQSettingsImpl : public BaseSettings<RabbitMQSettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) RabbitMQSettings##TYPE NAME = &RabbitMQSettingsImpl ::NAME;
|
||||
|
||||
namespace RabbitMQSetting
|
||||
{
|
||||
LIST_OF_RABBITMQ_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
}
|
||||
|
||||
#undef INITIALIZE_SETTING_EXTERN
|
||||
|
||||
RabbitMQSettings::RabbitMQSettings() : impl(std::make_unique<RabbitMQSettingsImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
RabbitMQSettings::RabbitMQSettings(const RabbitMQSettings & settings) : impl(std::make_unique<RabbitMQSettingsImpl>(*settings.impl))
|
||||
{
|
||||
}
|
||||
|
||||
RabbitMQSettings::RabbitMQSettings(RabbitMQSettings && settings) noexcept
|
||||
: impl(std::make_unique<RabbitMQSettingsImpl>(std::move(*settings.impl)))
|
||||
{
|
||||
}
|
||||
|
||||
RabbitMQSettings::~RabbitMQSettings() = default;
|
||||
|
||||
RABBITMQ_SETTINGS_SUPPORTED_TYPES(RabbitMQSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
|
||||
void RabbitMQSettings::loadFromQuery(ASTStorage & storage_def)
|
||||
{
|
||||
if (storage_def.settings)
|
||||
{
|
||||
try
|
||||
{
|
||||
applyChanges(storage_def.settings->changes);
|
||||
impl->applyChanges(storage_def.settings->changes);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
@ -36,4 +109,30 @@ void RabbitMQSettings::loadFromQuery(ASTStorage & storage_def)
|
||||
storage_def.set(storage_def.settings, settings_ast);
|
||||
}
|
||||
}
|
||||
|
||||
void RabbitMQSettings::loadFromNamedCollection(const MutableNamedCollectionPtr & named_collection)
|
||||
{
|
||||
for (const auto & setting : impl->all())
|
||||
{
|
||||
const auto & setting_name = setting.getName();
|
||||
if (named_collection->has(setting_name))
|
||||
impl->set(setting_name, named_collection->get<String>(setting_name));
|
||||
}
|
||||
}
|
||||
|
||||
SettingsChanges RabbitMQSettings::getFormatSettings() const
|
||||
{
|
||||
SettingsChanges values;
|
||||
|
||||
for (const auto & setting : *impl)
|
||||
{
|
||||
const auto & setting_name = setting.getName();
|
||||
|
||||
/// check for non-rabbitmq-related settings
|
||||
if (!setting_name.starts_with("rabbitmq_"))
|
||||
values.emplace_back(setting_name, setting.getValue());
|
||||
}
|
||||
|
||||
return values;
|
||||
}
|
||||
}
|
||||
|
@ -1,56 +1,60 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/FormatFactorySettingsDeclaration.h>
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
#include <Common/NamedCollections/NamedCollections_fwd.h>
|
||||
#include <Common/SettingsChanges.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTStorage;
|
||||
class ASTStorage;
|
||||
struct RabbitMQSettingsImpl;
|
||||
|
||||
/// List of available types supported in RabbitMQSettings object
|
||||
#define RABBITMQ_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, ArrowCompression) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, CapnProtoEnumComparingMode) \
|
||||
M(CLASS_NAME, Char) \
|
||||
M(CLASS_NAME, DateTimeInputFormat) \
|
||||
M(CLASS_NAME, DateTimeOutputFormat) \
|
||||
M(CLASS_NAME, DateTimeOverflowBehavior) \
|
||||
M(CLASS_NAME, Double) \
|
||||
M(CLASS_NAME, EscapingRule) \
|
||||
M(CLASS_NAME, Float) \
|
||||
M(CLASS_NAME, IdentifierQuotingRule) \
|
||||
M(CLASS_NAME, IdentifierQuotingStyle) \
|
||||
M(CLASS_NAME, Int64) \
|
||||
M(CLASS_NAME, IntervalOutputFormat) \
|
||||
M(CLASS_NAME, MsgPackUUIDRepresentation) \
|
||||
M(CLASS_NAME, ORCCompression) \
|
||||
M(CLASS_NAME, ParquetCompression) \
|
||||
M(CLASS_NAME, ParquetVersion) \
|
||||
M(CLASS_NAME, SchemaInferenceMode) \
|
||||
M(CLASS_NAME, StreamingHandleErrorMode) \
|
||||
M(CLASS_NAME, String) \
|
||||
M(CLASS_NAME, UInt64) \
|
||||
M(CLASS_NAME, UInt64Auto) \
|
||||
M(CLASS_NAME, URI)
|
||||
|
||||
#define RABBITMQ_RELATED_SETTINGS(M, ALIAS) \
|
||||
M(String, rabbitmq_host_port, "", "A host-port to connect to RabbitMQ server.", 0) \
|
||||
M(String, rabbitmq_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \
|
||||
M(String, rabbitmq_format, "", "The message format.", 0) \
|
||||
M(String, rabbitmq_exchange_type, "default", "The exchange type.", 0) \
|
||||
M(String, rabbitmq_routing_key_list, "5672", "A string of routing keys, separated by dots.", 0) \
|
||||
M(String, rabbitmq_schema, "", "Schema identifier (used by schema-based formats) for RabbitMQ engine", 0) \
|
||||
M(UInt64, rabbitmq_num_consumers, 1, "The number of consumer channels per table.", 0) \
|
||||
M(UInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \
|
||||
M(String, rabbitmq_queue_base, "", "Base for queue names to be able to reopen non-empty queues in case of failure.", 0) \
|
||||
M(Bool, rabbitmq_persistent, false, "For insert query messages will be made 'persistent', durable.", 0) \
|
||||
M(Bool, rabbitmq_secure, false, "Use SSL connection", 0) \
|
||||
M(String, rabbitmq_address, "", "Address for connection", 0) \
|
||||
M(UInt64, rabbitmq_skip_broken_messages, 0, "Skip at least this number of broken messages from RabbitMQ per block", 0) \
|
||||
M(UInt64, rabbitmq_max_block_size, 0, "Number of row collected before flushing data from RabbitMQ.", 0) \
|
||||
M(UInt64, rabbitmq_flush_interval_ms, 0, "Timeout for flushing data from RabbitMQ.", 0) \
|
||||
M(String, rabbitmq_vhost, "/", "RabbitMQ vhost.", 0) \
|
||||
M(String, rabbitmq_queue_settings_list, "", "A list of rabbitmq queue settings", 0) \
|
||||
M(UInt64, rabbitmq_empty_queue_backoff_start_ms, 10, "A minimum backoff point to reschedule read if the rabbitmq queue is empty", 0) \
|
||||
M(UInt64, rabbitmq_empty_queue_backoff_end_ms, 10000, "A maximum backoff point to reschedule read if the rabbitmq queue is empty", 0) \
|
||||
M(UInt64, rabbitmq_empty_queue_backoff_step_ms, 100, "A backoff step to reschedule read if the rabbitmq queue is empty", 0) \
|
||||
M(Bool, rabbitmq_queue_consume, false, "Use user-defined queues and do not make any RabbitMQ setup: declaring exchanges, queues, bindings", 0) \
|
||||
M(String, rabbitmq_username, "", "RabbitMQ username", 0) \
|
||||
M(String, rabbitmq_password, "", "RabbitMQ password", 0) \
|
||||
M(Bool, reject_unhandled_messages, false, "Allow messages to be rejected in case they cannot be processed. This also automatically implies if there is a x-deadletter-exchange queue setting added", 0) \
|
||||
M(Bool, rabbitmq_commit_on_select, false, "Commit messages when select query is made", 0) \
|
||||
M(UInt64, rabbitmq_max_rows_per_message, 1, "The maximum number of rows produced in one message for row-based formats.", 0) \
|
||||
M(StreamingHandleErrorMode, rabbitmq_handle_error_mode, StreamingHandleErrorMode::DEFAULT, "How to handle errors for RabbitMQ engine. Possible values: default (throw an exception after rabbitmq_skip_broken_messages broken messages), stream (save broken messages and errors in virtual columns _raw_message, _error).", 0) \
|
||||
RABBITMQ_SETTINGS_SUPPORTED_TYPES(RabbitMQSettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
#define OBSOLETE_RABBITMQ_SETTINGS(M, ALIAS) \
|
||||
MAKE_OBSOLETE(M, Char, rabbitmq_row_delimiter, '\0') \
|
||||
|
||||
|
||||
#define LIST_OF_RABBITMQ_SETTINGS(M, ALIAS) \
|
||||
RABBITMQ_RELATED_SETTINGS(M, ALIAS) \
|
||||
OBSOLETE_RABBITMQ_SETTINGS(M, ALIAS) \
|
||||
LIST_OF_ALL_FORMAT_SETTINGS(M, ALIAS) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(RabbitMQSettingsTraits, LIST_OF_RABBITMQ_SETTINGS)
|
||||
|
||||
struct RabbitMQSettings : public BaseSettings<RabbitMQSettingsTraits>
|
||||
struct RabbitMQSettings
|
||||
{
|
||||
RabbitMQSettings();
|
||||
RabbitMQSettings(const RabbitMQSettings & settings);
|
||||
RabbitMQSettings(RabbitMQSettings && settings) noexcept;
|
||||
~RabbitMQSettings();
|
||||
|
||||
RABBITMQ_SETTINGS_SUPPORTED_TYPES(RabbitMQSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void loadFromQuery(ASTStorage & storage_def);
|
||||
void loadFromNamedCollection(const MutableNamedCollectionPtr & named_collection);
|
||||
|
||||
SettingsChanges getFormatSettings() const;
|
||||
|
||||
private:
|
||||
std::unique_ptr<RabbitMQSettingsImpl> impl;
|
||||
};
|
||||
}
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <Storages/NamedCollectionsHelpers.h>
|
||||
#include <Storages/RabbitMQ/RabbitMQHandler.h>
|
||||
#include <Storages/RabbitMQ/RabbitMQProducer.h>
|
||||
#include <Storages/RabbitMQ/RabbitMQSettings.h>
|
||||
#include <Storages/RabbitMQ/RabbitMQSource.h>
|
||||
#include <Storages/RabbitMQ/StorageRabbitMQ.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
@ -33,6 +34,8 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/setThreadName.h>
|
||||
|
||||
#include <base/range.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace Setting
|
||||
@ -43,6 +46,37 @@ namespace Setting
|
||||
extern const SettingsBool stream_like_engine_allow_direct_select;
|
||||
}
|
||||
|
||||
namespace RabbitMQSetting
|
||||
{
|
||||
extern const RabbitMQSettingsString rabbitmq_address;
|
||||
extern const RabbitMQSettingsBool rabbitmq_commit_on_select;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_empty_queue_backoff_end_ms;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_empty_queue_backoff_start_ms;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_empty_queue_backoff_step_ms;
|
||||
extern const RabbitMQSettingsString rabbitmq_exchange_name;
|
||||
extern const RabbitMQSettingsString rabbitmq_exchange_type;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_flush_interval_ms;
|
||||
extern const RabbitMQSettingsString rabbitmq_format;
|
||||
extern const RabbitMQSettingsStreamingHandleErrorMode rabbitmq_handle_error_mode;
|
||||
extern const RabbitMQSettingsString rabbitmq_host_port;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_max_block_size;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_max_rows_per_message;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_num_consumers;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_num_queues;
|
||||
extern const RabbitMQSettingsString rabbitmq_password;
|
||||
extern const RabbitMQSettingsBool rabbitmq_persistent;
|
||||
extern const RabbitMQSettingsString rabbitmq_queue_base;
|
||||
extern const RabbitMQSettingsBool rabbitmq_queue_consume;
|
||||
extern const RabbitMQSettingsString rabbitmq_queue_settings_list;
|
||||
extern const RabbitMQSettingsString rabbitmq_routing_key_list;
|
||||
extern const RabbitMQSettingsString rabbitmq_schema;
|
||||
extern const RabbitMQSettingsBool rabbitmq_secure;
|
||||
extern const RabbitMQSettingsUInt64 rabbitmq_skip_broken_messages;
|
||||
extern const RabbitMQSettingsString rabbitmq_username;
|
||||
extern const RabbitMQSettingsString rabbitmq_vhost;
|
||||
extern const RabbitMQSettingsBool reject_unhandled_messages;
|
||||
}
|
||||
|
||||
static const uint32_t QUEUE_SIZE = 100000;
|
||||
static const auto MAX_FAILED_READ_ATTEMPTS = 10;
|
||||
static const auto RESCHEDULE_MS = 500;
|
||||
@ -84,26 +118,26 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
: IStorage(table_id_)
|
||||
, WithContext(context_->getGlobalContext())
|
||||
, rabbitmq_settings(std::move(rabbitmq_settings_))
|
||||
, exchange_name(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_exchange_name))
|
||||
, format_name(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_format))
|
||||
, exchange_type(defineExchangeType(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_exchange_type)))
|
||||
, routing_keys(parseSettings(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_routing_key_list)))
|
||||
, schema_name(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_schema))
|
||||
, num_consumers(rabbitmq_settings->rabbitmq_num_consumers.value)
|
||||
, num_queues(rabbitmq_settings->rabbitmq_num_queues.value)
|
||||
, queue_base(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_queue_base))
|
||||
, queue_settings_list(parseSettings(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_queue_settings_list)))
|
||||
, max_rows_per_message(rabbitmq_settings->rabbitmq_max_rows_per_message)
|
||||
, exchange_name(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_exchange_name]))
|
||||
, format_name(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_format]))
|
||||
, exchange_type(defineExchangeType(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_exchange_type])))
|
||||
, routing_keys(parseSettings(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_routing_key_list])))
|
||||
, schema_name(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_schema]))
|
||||
, num_consumers((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_num_consumers].value)
|
||||
, num_queues((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_num_queues].value)
|
||||
, queue_base(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_queue_base]))
|
||||
, queue_settings_list(parseSettings(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_queue_settings_list])))
|
||||
, max_rows_per_message((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_max_rows_per_message])
|
||||
, log(getLogger("StorageRabbitMQ (" + table_id_.table_name + ")"))
|
||||
, persistent(rabbitmq_settings->rabbitmq_persistent.value)
|
||||
, use_user_setup(rabbitmq_settings->rabbitmq_queue_consume.value)
|
||||
, persistent((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_persistent].value)
|
||||
, use_user_setup((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_queue_consume].value)
|
||||
, hash_exchange(num_consumers > 1 || num_queues > 1)
|
||||
, semaphore(0, static_cast<int>(num_consumers))
|
||||
, unique_strbase(getRandomName())
|
||||
, queue_size(std::max(QUEUE_SIZE, static_cast<uint32_t>(getMaxBlockSize())))
|
||||
, milliseconds_to_wait(rabbitmq_settings->rabbitmq_empty_queue_backoff_start_ms)
|
||||
, milliseconds_to_wait((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_empty_queue_backoff_start_ms])
|
||||
{
|
||||
reject_unhandled_messages = rabbitmq_settings->reject_unhandled_messages
|
||||
reject_unhandled_messages = (*rabbitmq_settings)[RabbitMQSetting::reject_unhandled_messages]
|
||||
|| queue_settings_list.end() !=
|
||||
std::find_if(queue_settings_list.begin(), queue_settings_list.end(),
|
||||
[](const String & name) { return name.starts_with(deadletter_exchange_setting); });
|
||||
@ -111,11 +145,11 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
const auto & config = getContext()->getConfigRef();
|
||||
|
||||
std::pair<String, UInt16> parsed_address;
|
||||
auto setting_rabbitmq_username = rabbitmq_settings->rabbitmq_username.value;
|
||||
auto setting_rabbitmq_password = rabbitmq_settings->rabbitmq_password.value;
|
||||
auto setting_rabbitmq_username = (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_username].value;
|
||||
auto setting_rabbitmq_password = (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_password].value;
|
||||
String username, password;
|
||||
|
||||
if (rabbitmq_settings->rabbitmq_host_port.changed)
|
||||
if ((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_host_port].changed)
|
||||
{
|
||||
username = setting_rabbitmq_username.empty() ? config.getString("rabbitmq.username", "") : setting_rabbitmq_username;
|
||||
password = setting_rabbitmq_password.empty() ? config.getString("rabbitmq.password", "") : setting_rabbitmq_password;
|
||||
@ -124,7 +158,7 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"No username or password. They can be specified either in config or in storage settings");
|
||||
|
||||
parsed_address = parseAddress(getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_host_port), 5672);
|
||||
parsed_address = parseAddress(getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_host_port]), 5672);
|
||||
if (parsed_address.first.empty())
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
@ -132,7 +166,7 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
|
||||
context_->getRemoteHostFilter().checkHostAndPort(parsed_address.first, toString(parsed_address.second));
|
||||
}
|
||||
else if (!rabbitmq_settings->rabbitmq_address.changed)
|
||||
else if (!(*rabbitmq_settings)[RabbitMQSetting::rabbitmq_address].changed)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "RabbitMQ requires either `rabbitmq_host_port` or `rabbitmq_address` setting");
|
||||
|
||||
configuration =
|
||||
@ -141,9 +175,9 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
.port = parsed_address.second,
|
||||
.username = username,
|
||||
.password = password,
|
||||
.vhost = config.getString("rabbitmq.vhost", getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_vhost)),
|
||||
.secure = rabbitmq_settings->rabbitmq_secure.value,
|
||||
.connection_string = getContext()->getMacros()->expand(rabbitmq_settings->rabbitmq_address)
|
||||
.vhost = config.getString("rabbitmq.vhost", getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_vhost])),
|
||||
.secure = (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_secure].value,
|
||||
.connection_string = getContext()->getMacros()->expand((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_address])
|
||||
};
|
||||
|
||||
if (configuration.secure)
|
||||
@ -156,7 +190,7 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
storage_metadata.setColumns(columns_);
|
||||
storage_metadata.setComment(comment);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
setVirtuals(createVirtuals(rabbitmq_settings->rabbitmq_handle_error_mode));
|
||||
setVirtuals(createVirtuals((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_handle_error_mode]));
|
||||
|
||||
rabbitmq_context = addSettings(getContext());
|
||||
rabbitmq_context->makeQueryContext();
|
||||
@ -212,6 +246,8 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
init_task->deactivate();
|
||||
}
|
||||
|
||||
StorageRabbitMQ::~StorageRabbitMQ() = default;
|
||||
|
||||
VirtualColumnsDescription StorageRabbitMQ::createVirtuals(StreamingHandleErrorMode handle_error_mode)
|
||||
{
|
||||
VirtualColumnsDescription desc;
|
||||
@ -280,8 +316,8 @@ ContextMutablePtr StorageRabbitMQ::addSettings(ContextPtr local_context) const
|
||||
auto modified_context = Context::createCopy(local_context);
|
||||
modified_context->setSetting("input_format_skip_unknown_fields", true);
|
||||
modified_context->setSetting("input_format_allow_errors_ratio", 0.);
|
||||
if (rabbitmq_settings->rabbitmq_handle_error_mode == StreamingHandleErrorMode::DEFAULT)
|
||||
modified_context->setSetting("input_format_allow_errors_num", rabbitmq_settings->rabbitmq_skip_broken_messages.value);
|
||||
if ((*rabbitmq_settings)[RabbitMQSetting::rabbitmq_handle_error_mode] == StreamingHandleErrorMode::DEFAULT)
|
||||
modified_context->setSetting("input_format_allow_errors_num", (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_skip_broken_messages].value);
|
||||
else
|
||||
modified_context->setSetting("input_format_allow_errors_num", Field(0));
|
||||
|
||||
@ -291,14 +327,8 @@ ContextMutablePtr StorageRabbitMQ::addSettings(ContextPtr local_context) const
|
||||
if (!schema_name.empty())
|
||||
modified_context->setSetting("format_schema", schema_name);
|
||||
|
||||
for (const auto & setting : *rabbitmq_settings)
|
||||
{
|
||||
const auto & setting_name = setting.getName();
|
||||
|
||||
/// check for non-rabbitmq-related settings
|
||||
if (!setting_name.starts_with("rabbitmq_"))
|
||||
modified_context->setSetting(setting_name, setting.getValue());
|
||||
}
|
||||
/// check for non-rabbitmq-related settings
|
||||
modified_context->applySettingsChanges(rabbitmq_settings->getFormatSettings());
|
||||
|
||||
return modified_context;
|
||||
}
|
||||
@ -396,8 +426,8 @@ void StorageRabbitMQ::deactivateTask(BackgroundSchedulePool::TaskHolder & task,
|
||||
|
||||
size_t StorageRabbitMQ::getMaxBlockSize() const
|
||||
{
|
||||
return rabbitmq_settings->rabbitmq_max_block_size.changed
|
||||
? rabbitmq_settings->rabbitmq_max_block_size.value
|
||||
return (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_max_block_size].changed
|
||||
? (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_max_block_size].value
|
||||
: (getContext()->getSettingsRef()[Setting::max_insert_block_size].value / num_consumers);
|
||||
}
|
||||
|
||||
@ -773,16 +803,16 @@ void StorageRabbitMQ::read(
|
||||
Pipes pipes;
|
||||
pipes.reserve(num_created_consumers);
|
||||
|
||||
uint64_t max_execution_time_ms = rabbitmq_settings->rabbitmq_flush_interval_ms.changed
|
||||
? rabbitmq_settings->rabbitmq_flush_interval_ms
|
||||
uint64_t max_execution_time_ms = (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_flush_interval_ms].changed
|
||||
? (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_flush_interval_ms]
|
||||
: static_cast<UInt64>(getContext()->getSettingsRef()[Setting::stream_flush_interval_ms].totalMilliseconds());
|
||||
|
||||
for (size_t i = 0; i < num_created_consumers; ++i)
|
||||
{
|
||||
auto rabbit_source = std::make_shared<RabbitMQSource>(
|
||||
*this, storage_snapshot, modified_context, column_names, /* max_block_size */1,
|
||||
max_execution_time_ms, rabbitmq_settings->rabbitmq_handle_error_mode, reject_unhandled_messages,
|
||||
/* ack_in_suffix */rabbitmq_settings->rabbitmq_commit_on_select, log);
|
||||
max_execution_time_ms, (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_handle_error_mode], reject_unhandled_messages,
|
||||
/* ack_in_suffix */(*rabbitmq_settings)[RabbitMQSetting::rabbitmq_commit_on_select], log);
|
||||
|
||||
auto converting_dag = ActionsDAG::makeConvertingActions(
|
||||
rabbit_source->getPort().getHeader().getColumnsWithTypeAndName(),
|
||||
@ -1041,8 +1071,8 @@ void StorageRabbitMQ::streamingToViewsFunc()
|
||||
else
|
||||
{
|
||||
/// Reschedule with backoff.
|
||||
if (milliseconds_to_wait < rabbitmq_settings->rabbitmq_empty_queue_backoff_end_ms)
|
||||
milliseconds_to_wait += rabbitmq_settings->rabbitmq_empty_queue_backoff_step_ms;
|
||||
if (milliseconds_to_wait < (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_empty_queue_backoff_end_ms])
|
||||
milliseconds_to_wait += (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_empty_queue_backoff_step_ms];
|
||||
|
||||
LOG_DEBUG(log, "Rescheduling background streaming process in {}", milliseconds_to_wait);
|
||||
streaming_task->scheduleAfter(milliseconds_to_wait);
|
||||
@ -1089,7 +1119,7 @@ void StorageRabbitMQ::streamToViewsImpl()
|
||||
break;
|
||||
}
|
||||
|
||||
milliseconds_to_wait = rabbitmq_settings->rabbitmq_empty_queue_backoff_start_ms;
|
||||
milliseconds_to_wait = (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_empty_queue_backoff_start_ms];
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1110,15 +1140,15 @@ bool StorageRabbitMQ::tryStreamToViews()
|
||||
sources.reserve(num_created_consumers);
|
||||
pipes.reserve(num_created_consumers);
|
||||
|
||||
uint64_t max_execution_time_ms = rabbitmq_settings->rabbitmq_flush_interval_ms.changed
|
||||
? rabbitmq_settings->rabbitmq_flush_interval_ms
|
||||
uint64_t max_execution_time_ms = (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_flush_interval_ms].changed
|
||||
? (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_flush_interval_ms]
|
||||
: static_cast<UInt64>(getContext()->getSettingsRef()[Setting::stream_flush_interval_ms].totalMilliseconds());
|
||||
|
||||
for (size_t i = 0; i < num_created_consumers; ++i)
|
||||
{
|
||||
auto source = std::make_shared<RabbitMQSource>(
|
||||
*this, storage_snapshot, rabbitmq_context, Names{}, block_size,
|
||||
max_execution_time_ms, rabbitmq_settings->rabbitmq_handle_error_mode,
|
||||
max_execution_time_ms, (*rabbitmq_settings)[RabbitMQSetting::rabbitmq_handle_error_mode],
|
||||
reject_unhandled_messages, /* ack_in_suffix */false, log);
|
||||
|
||||
sources.emplace_back(source);
|
||||
@ -1274,26 +1304,19 @@ void registerStorageRabbitMQ(StorageFactory & factory)
|
||||
auto rabbitmq_settings = std::make_unique<RabbitMQSettings>();
|
||||
|
||||
if (auto named_collection = tryGetNamedCollectionWithOverrides(args.engine_args, args.getLocalContext()))
|
||||
{
|
||||
for (const auto & setting : rabbitmq_settings->all())
|
||||
{
|
||||
const auto & setting_name = setting.getName();
|
||||
if (named_collection->has(setting_name))
|
||||
rabbitmq_settings->set(setting_name, named_collection->get<String>(setting_name));
|
||||
}
|
||||
}
|
||||
rabbitmq_settings->loadFromNamedCollection(named_collection);
|
||||
else if (!args.storage_def->settings)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "RabbitMQ engine must have settings");
|
||||
|
||||
if (args.storage_def->settings)
|
||||
rabbitmq_settings->loadFromQuery(*args.storage_def);
|
||||
|
||||
if (!rabbitmq_settings->rabbitmq_host_port.changed
|
||||
&& !rabbitmq_settings->rabbitmq_address.changed)
|
||||
if (!(*rabbitmq_settings)[RabbitMQSetting::rabbitmq_host_port].changed
|
||||
&& !(*rabbitmq_settings)[RabbitMQSetting::rabbitmq_address].changed)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"You must specify either `rabbitmq_host_port` or `rabbitmq_address` settings");
|
||||
|
||||
if (!rabbitmq_settings->rabbitmq_format.changed)
|
||||
if (!(*rabbitmq_settings)[RabbitMQSetting::rabbitmq_format].changed)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "You must specify `rabbitmq_format` setting");
|
||||
|
||||
return std::make_shared<StorageRabbitMQ>(args.table_id, args.getContext(), args.columns, args.comment, std::move(rabbitmq_settings), args.mode);
|
||||
|
@ -1,12 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Poco/Semaphore.h>
|
||||
#include <mutex>
|
||||
#include <atomic>
|
||||
#include <Storages/RabbitMQ/RabbitMQConsumer.h>
|
||||
#include <Storages/RabbitMQ/RabbitMQSettings.h>
|
||||
#include <Storages/RabbitMQ/RabbitMQConnection.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <amqpcpp/libuv.h>
|
||||
@ -16,7 +16,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct RabbitMQSettings;
|
||||
using RabbitMQConsumerPtr = std::shared_ptr<RabbitMQConsumer>;
|
||||
|
||||
class StorageRabbitMQ final: public IStorage, WithContext
|
||||
@ -30,6 +30,8 @@ public:
|
||||
std::unique_ptr<RabbitMQSettings> rabbitmq_settings_,
|
||||
LoadingStrictnessLevel mode);
|
||||
|
||||
~StorageRabbitMQ() override;
|
||||
|
||||
std::string getName() const override { return "RabbitMQ"; }
|
||||
|
||||
bool noPushingToViews() const override { return true; }
|
||||
|
@ -35,6 +35,11 @@ namespace Setting
|
||||
extern const SettingsUInt64 min_insert_block_size_rows;
|
||||
}
|
||||
|
||||
namespace RocksDBSetting
|
||||
{
|
||||
extern const RocksDBSettingsUInt64 bulk_insert_block_size;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ROCKSDB_ERROR;
|
||||
@ -91,7 +96,7 @@ EmbeddedRocksDBBulkSink::EmbeddedRocksDBBulkSink(
|
||||
|
||||
serializations = getHeader().getSerializations();
|
||||
min_block_size_rows
|
||||
= std::max(storage.getSettings().bulk_insert_block_size, getContext()->getSettingsRef()[Setting::min_insert_block_size_rows]);
|
||||
= std::max(storage.getSettings()[RocksDBSetting::bulk_insert_block_size], getContext()->getSettingsRef()[Setting::min_insert_block_size_rows]);
|
||||
|
||||
/// If max_insert_threads > 1 we may have multiple EmbeddedRocksDBBulkSink and getContext()->getCurrentQueryId() is not guarantee to
|
||||
/// to have a distinct path. Also we cannot use query id as directory name here, because it could be defined by user and not suitable
|
||||
|
@ -1,6 +1,8 @@
|
||||
#include "RocksDBSettings.h"
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Storages/RocksDB/RocksDBSettings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -10,17 +12,59 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_SETTING;
|
||||
}
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(RockDBSettingsTraits, LIST_OF_ROCKSDB_SETTINGS)
|
||||
/** StorageEmbeddedRocksdb table settings
|
||||
*/
|
||||
#define LIST_OF_ROCKSDB_SETTINGS(M, ALIAS) \
|
||||
M(Bool, optimize_for_bulk_insert, true, "Table is optimized for bulk insertions (insert pipeline will create SST files and import to rocksdb database instead of writing to memtables)", 0) \
|
||||
M(UInt64, bulk_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "Size of block for bulk insert, if it's smaller than query setting min_insert_block_size_rows then it will be overridden by min_insert_block_size_rows", 0) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(RocksDBSettingsTraits, LIST_OF_ROCKSDB_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(RocksDBSettingsTraits, LIST_OF_ROCKSDB_SETTINGS)
|
||||
|
||||
struct RocksDBSettingsImpl : public BaseSettings<RocksDBSettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) RocksDBSettings##TYPE NAME = &RocksDBSettingsImpl ::NAME;
|
||||
|
||||
namespace RocksDBSetting
|
||||
{
|
||||
LIST_OF_ROCKSDB_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
}
|
||||
|
||||
#undef INITIALIZE_SETTING_EXTERN
|
||||
|
||||
|
||||
void RocksDBSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr /*context*/)
|
||||
RocksDBSettings::RocksDBSettings() : impl(std::make_unique<RocksDBSettingsImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
RocksDBSettings::RocksDBSettings(const RocksDBSettings & settings) : impl(std::make_unique<RocksDBSettingsImpl>(*settings.impl))
|
||||
{
|
||||
}
|
||||
|
||||
RocksDBSettings::RocksDBSettings(RocksDBSettings && settings) noexcept
|
||||
: impl(std::make_unique<RocksDBSettingsImpl>(std::move(*settings.impl)))
|
||||
{
|
||||
}
|
||||
|
||||
RocksDBSettings::~RocksDBSettings() = default;
|
||||
|
||||
ROCKSDB_SETTINGS_SUPPORTED_TYPES(RocksDBSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void RocksDBSettings::applyChanges(const SettingsChanges & changes)
|
||||
{
|
||||
impl->applyChanges(changes);
|
||||
}
|
||||
|
||||
void RocksDBSettings::loadFromQuery(const ASTStorage & storage_def)
|
||||
{
|
||||
if (storage_def.settings)
|
||||
{
|
||||
try
|
||||
{
|
||||
auto changes = storage_def.settings->changes;
|
||||
applyChanges(changes);
|
||||
impl->applyChanges(changes);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
@ -30,12 +74,4 @@ void RocksDBSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr /*conte
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<String> RocksDBSettings::getAllRegisteredNames() const
|
||||
{
|
||||
std::vector<String> all_settings;
|
||||
for (const auto & setting_field : all())
|
||||
all_settings.push_back(setting_field.getName());
|
||||
return all_settings;
|
||||
}
|
||||
}
|
||||
|
@ -1,39 +1,34 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <base/unit.h>
|
||||
#include <Common/NamePrompter.h>
|
||||
|
||||
|
||||
namespace Poco::Util
|
||||
{
|
||||
class AbstractConfiguration;
|
||||
}
|
||||
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTStorage;
|
||||
struct Settings;
|
||||
struct RocksDBSettingsImpl;
|
||||
class SettingsChanges;
|
||||
|
||||
/// List of available types supported in RocksDBSettings object
|
||||
#define ROCKSDB_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, UInt64)
|
||||
|
||||
/** StorageEmbeddedRocksdb table settings
|
||||
*/
|
||||
ROCKSDB_SETTINGS_SUPPORTED_TYPES(RocksDBSettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
#define ROCKSDB_SETTINGS(M, ALIAS) \
|
||||
M(Bool, optimize_for_bulk_insert, true, "Table is optimized for bulk insertions (insert pipeline will create SST files and import to rocksdb database instead of writing to memtables)", 0) \
|
||||
M(UInt64, bulk_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "Size of block for bulk insert, if it's smaller than query setting min_insert_block_size_rows then it will be overridden by min_insert_block_size_rows", 0) \
|
||||
|
||||
#define LIST_OF_ROCKSDB_SETTINGS(M, ALIAS) ROCKSDB_SETTINGS(M, ALIAS)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(RockDBSettingsTraits, LIST_OF_ROCKSDB_SETTINGS)
|
||||
|
||||
struct RocksDBSettings : public BaseSettings<RockDBSettingsTraits>, public IHints<2>
|
||||
struct RocksDBSettings
|
||||
{
|
||||
void loadFromQuery(ASTStorage & storage_def, ContextPtr context);
|
||||
std::vector<String> getAllRegisteredNames() const override;
|
||||
};
|
||||
RocksDBSettings();
|
||||
RocksDBSettings(const RocksDBSettings & settings);
|
||||
RocksDBSettings(RocksDBSettings && settings) noexcept;
|
||||
~RocksDBSettings();
|
||||
|
||||
ROCKSDB_SETTINGS_SUPPORTED_TYPES(RocksDBSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void applyChanges(const SettingsChanges & changes);
|
||||
void loadFromQuery(const ASTStorage & storage_def);
|
||||
|
||||
private:
|
||||
std::unique_ptr<RocksDBSettingsImpl> impl;
|
||||
};
|
||||
}
|
||||
|
@ -57,6 +57,11 @@ namespace Setting
|
||||
extern const SettingsBool optimize_trivial_approximate_count_query;
|
||||
}
|
||||
|
||||
namespace RocksDBSetting
|
||||
{
|
||||
extern const RocksDBSettingsBool optimize_for_bulk_insert;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
@ -663,7 +668,7 @@ void ReadFromEmbeddedRocksDB::applyFilters(ActionDAGNodes added_filter_nodes)
|
||||
SinkToStoragePtr StorageEmbeddedRocksDB::write(
|
||||
const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context, bool /*async_insert*/)
|
||||
{
|
||||
if (getSettings().optimize_for_bulk_insert)
|
||||
if (getSettings()[RocksDBSetting::optimize_for_bulk_insert])
|
||||
{
|
||||
LOG_DEBUG(log, "Using bulk insert");
|
||||
return std::make_shared<EmbeddedRocksDBBulkSink>(query_context, *this, metadata_snapshot);
|
||||
@ -710,7 +715,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageEmbeddedRocksDB must require one column in primary key");
|
||||
}
|
||||
auto settings = std::make_unique<RocksDBSettings>();
|
||||
settings->loadFromQuery(*args.storage_def, args.getContext());
|
||||
settings->loadFromQuery(*args.storage_def);
|
||||
if (args.storage_def->settings)
|
||||
metadata.settings_changes = args.storage_def->settings->ptr();
|
||||
else
|
||||
@ -720,7 +725,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
/// SETTING queries. So we just add a setting with its default value.
|
||||
auto settings_changes = std::make_shared<ASTSetQuery>();
|
||||
settings_changes->is_standalone = false;
|
||||
settings_changes->changes.insertSetting("optimize_for_bulk_insert", settings->optimize_for_bulk_insert.value);
|
||||
settings_changes->changes.insertSetting("optimize_for_bulk_insert", (*settings)[RocksDBSetting::optimize_for_bulk_insert].value);
|
||||
metadata.settings_changes = settings_changes;
|
||||
}
|
||||
return std::make_shared<StorageEmbeddedRocksDB>(args.table_id, args.relative_data_path, metadata, args.mode, args.getContext(), std::move(settings), primary_key_names[0], ttl, std::move(rocksdb_dir), read_only);
|
||||
|
@ -53,6 +53,11 @@ namespace ServerSetting
|
||||
extern const ServerSettingsUInt64 max_materialized_views_count_for_table;
|
||||
}
|
||||
|
||||
namespace RefreshSetting
|
||||
{
|
||||
extern const RefreshSettingsBool all_replicas;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
@ -177,7 +182,7 @@ StorageMaterializedView::StorageMaterializedView(
|
||||
RefreshSettings s;
|
||||
if (query.refresh_strategy->settings)
|
||||
s.applyChanges(query.refresh_strategy->settings->changes);
|
||||
refresh_coordinated = !s.all_replicas;
|
||||
refresh_coordinated = !s[RefreshSetting::all_replicas];
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -179,9 +179,9 @@ StorageTimeSeries::StorageTimeSeries(
|
||||
StorageTimeSeries::~StorageTimeSeries() = default;
|
||||
|
||||
|
||||
TimeSeriesSettings StorageTimeSeries::getStorageSettings() const
|
||||
const TimeSeriesSettings & StorageTimeSeries::getStorageSettings() const
|
||||
{
|
||||
return *getStorageSettingsPtr();
|
||||
return *storage_settings;
|
||||
}
|
||||
|
||||
void StorageTimeSeries::startup()
|
||||
|
@ -41,8 +41,7 @@ public:
|
||||
|
||||
std::string getName() const override { return "TimeSeries"; }
|
||||
|
||||
TimeSeriesSettings getStorageSettings() const;
|
||||
TimeSeriesSettingsPtr getStorageSettingsPtr() const { return storage_settings; }
|
||||
const TimeSeriesSettings & getStorageSettings() const;
|
||||
|
||||
StorageID getTargetTableId(ViewTarget::Kind target_kind) const;
|
||||
StoragePtr getTargetTable(ViewTarget::Kind target_kind, const ContextPtr & local_context) const;
|
||||
|
@ -31,6 +31,12 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace TimeSeriesSetting
|
||||
{
|
||||
extern const TimeSeriesSettingsBool filter_by_min_time_and_max_time;
|
||||
extern const TimeSeriesSettingsMap tags_to_columns;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_REQUEST_PARAMETER;
|
||||
@ -160,7 +166,7 @@ namespace
|
||||
std::unordered_map<String, String> makeColumnNameByTagNameMap(const TimeSeriesSettings & storage_settings)
|
||||
{
|
||||
std::unordered_map<String, String> res;
|
||||
const Map & tags_to_columns = storage_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = storage_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -196,7 +202,7 @@ namespace
|
||||
exp_list->children.push_back(
|
||||
makeASTColumn(tags_table_id, TimeSeriesColumnNames::MetricName));
|
||||
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -255,7 +261,7 @@ namespace
|
||||
|
||||
/// WHERE <filter>
|
||||
if (auto where = makeASTFilterForReadingTimeSeries(label_matcher, min_timestamp_ms, max_timestamp_ms, data_table_id, tags_table_id,
|
||||
column_name_by_tag_name, time_series_settings.filter_by_min_time_and_max_time))
|
||||
column_name_by_tag_name, time_series_settings[TimeSeriesSetting::filter_by_min_time_and_max_time]))
|
||||
{
|
||||
select_query->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where));
|
||||
}
|
||||
@ -267,7 +273,7 @@ namespace
|
||||
exp_list->children.push_back(
|
||||
makeASTColumn(tags_table_id, TimeSeriesColumnNames::MetricName));
|
||||
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -329,7 +335,7 @@ namespace
|
||||
|
||||
/// Columns corresponding to specific tags specified in the "tags_to_columns" setting.
|
||||
std::unordered_map<String, const IColumn *> column_by_tag_name;
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -447,12 +453,12 @@ void PrometheusRemoteReadProtocol::readTimeSeries(google::protobuf::RepeatedPtrF
|
||||
out_time_series.Clear();
|
||||
|
||||
auto time_series_storage_id = time_series_storage->getStorageID();
|
||||
auto time_series_settings = time_series_storage->getStorageSettingsPtr();
|
||||
const auto & time_series_settings = time_series_storage->getStorageSettings();
|
||||
auto data_table_id = time_series_storage->getTargetTableId(ViewTarget::Data);
|
||||
auto tags_table_id = time_series_storage->getTargetTableId(ViewTarget::Tags);
|
||||
|
||||
ASTPtr select_query = buildSelectQueryForReadingTimeSeries(
|
||||
start_timestamp_ms, end_timestamp_ms, label_matcher, *time_series_settings, data_table_id, tags_table_id);
|
||||
start_timestamp_ms, end_timestamp_ms, label_matcher, time_series_settings, data_table_id, tags_table_id);
|
||||
|
||||
LOG_TRACE(log, "{}: Executing query {}",
|
||||
time_series_storage_id.getNameForLogs(), select_query);
|
||||
@ -468,7 +474,7 @@ void PrometheusRemoteReadProtocol::readTimeSeries(google::protobuf::RepeatedPtrF
|
||||
time_series_storage_id.getNameForLogs(), block.columns(), block.rows());
|
||||
|
||||
if (block)
|
||||
convertBlockToProtobuf(std::move(block), out_time_series, time_series_storage_id, *time_series_settings);
|
||||
convertBlockToProtobuf(std::move(block), out_time_series, time_series_storage_id, time_series_settings);
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "{}: {} time series read",
|
||||
|
@ -32,6 +32,13 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace TimeSeriesSetting
|
||||
{
|
||||
extern const TimeSeriesSettingsBool store_min_time_and_max_time;
|
||||
extern const TimeSeriesSettingsMap tags_to_columns;
|
||||
extern const TimeSeriesSettingsBool use_all_tags_column_to_generate_id;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TIME_SERIES_TAGS;
|
||||
@ -261,7 +268,7 @@ namespace
|
||||
|
||||
/// Columns corresponding to specific tags specified in the "tags_to_columns" setting.
|
||||
std::unordered_map<String, IColumn *> columns_by_tag_name;
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -286,7 +293,7 @@ namespace
|
||||
IColumn * all_tags_names = nullptr;
|
||||
IColumn * all_tags_values = nullptr;
|
||||
IColumn::Offsets * all_tags_offsets = nullptr;
|
||||
if (time_series_settings.use_all_tags_column_to_generate_id)
|
||||
if (time_series_settings[TimeSeriesSetting::use_all_tags_column_to_generate_id])
|
||||
{
|
||||
const auto & all_tags_description = get_column_description(TimeSeriesColumnNames::AllTags);
|
||||
validator.validateColumnForTagsMap(all_tags_description);
|
||||
@ -301,7 +308,7 @@ namespace
|
||||
IColumn * max_time_column = nullptr;
|
||||
UInt32 min_time_scale = 0;
|
||||
UInt32 max_time_scale = 0;
|
||||
if (time_series_settings.store_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::store_min_time_and_max_time])
|
||||
{
|
||||
const auto & min_time_description = get_column_description(TimeSeriesColumnNames::MinTime);
|
||||
const auto & max_time_description = get_column_description(TimeSeriesColumnNames::MaxTime);
|
||||
@ -336,7 +343,7 @@ namespace
|
||||
}
|
||||
else
|
||||
{
|
||||
if (time_series_settings.use_all_tags_column_to_generate_id)
|
||||
if (time_series_settings[TimeSeriesSetting::use_all_tags_column_to_generate_id])
|
||||
{
|
||||
all_tags_names->insertData(tag_name.data(), tag_name.length());
|
||||
all_tags_values->insertData(tag_value.data(), tag_value.length());
|
||||
@ -359,10 +366,10 @@ namespace
|
||||
|
||||
tags_offsets.push_back(tags_names.size());
|
||||
|
||||
if (time_series_settings.use_all_tags_column_to_generate_id)
|
||||
if (time_series_settings[TimeSeriesSetting::use_all_tags_column_to_generate_id])
|
||||
all_tags_offsets->push_back(all_tags_names->size());
|
||||
|
||||
if (time_series_settings.store_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::store_min_time_and_max_time])
|
||||
{
|
||||
auto [min_time, max_time] = findMinTimeAndMaxTime(element.samples());
|
||||
min_time_column->insert(scaleTimestamp(min_time, min_time_scale));
|
||||
@ -571,9 +578,9 @@ void PrometheusRemoteWriteProtocol::writeTimeSeries(const google::protobuf::Repe
|
||||
time_series_storage_id.getNameForLogs(), time_series.size());
|
||||
|
||||
auto time_series_storage_metadata = time_series_storage->getInMemoryMetadataPtr();
|
||||
auto time_series_settings = time_series_storage->getStorageSettingsPtr();
|
||||
const auto & time_series_settings = time_series_storage->getStorageSettings();
|
||||
|
||||
auto blocks = toBlocks(time_series, getContext(), time_series_storage_id, *time_series_storage_metadata, *time_series_settings);
|
||||
auto blocks = toBlocks(time_series, getContext(), time_series_storage_id, *time_series_storage_metadata, time_series_settings);
|
||||
insertToTargetTables(std::move(blocks), *time_series_storage, getContext(), log.get());
|
||||
|
||||
LOG_TRACE(log, "{}: {} time series written",
|
||||
@ -588,9 +595,9 @@ void PrometheusRemoteWriteProtocol::writeMetricsMetadata(const google::protobuf:
|
||||
time_series_storage_id.getNameForLogs(), metrics_metadata.size());
|
||||
|
||||
auto time_series_storage_metadata = time_series_storage->getInMemoryMetadataPtr();
|
||||
auto time_series_settings = time_series_storage->getStorageSettingsPtr();
|
||||
const auto & time_series_settings = time_series_storage->getStorageSettings();
|
||||
|
||||
auto blocks = toBlocks(metrics_metadata, time_series_storage_id, *time_series_storage_metadata, *time_series_settings);
|
||||
auto blocks = toBlocks(metrics_metadata, time_series_storage_id, *time_series_storage_metadata, time_series_settings);
|
||||
insertToTargetTables(std::move(blocks), *time_series_storage, getContext(), log.get());
|
||||
|
||||
LOG_TRACE(log, "{}: {} metrics metadata written",
|
||||
|
@ -12,6 +12,11 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace TimeSeriesSetting
|
||||
{
|
||||
extern const TimeSeriesSettingsMap tags_to_columns;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
@ -63,7 +68,7 @@ void TimeSeriesColumnsValidator::validateColumnsImpl(const ColumnsDescription &
|
||||
/// Validate columns for the "tags" table.
|
||||
validateColumnForMetricName(get_column_description(TimeSeriesColumnNames::MetricName));
|
||||
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -127,7 +132,7 @@ void TimeSeriesColumnsValidator::validateTargetColumnsImpl(ViewTarget::Kind targ
|
||||
{
|
||||
validateColumnForMetricName(get_column_description(TimeSeriesColumnNames::MetricName));
|
||||
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
|
@ -15,6 +15,14 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace TimeSeriesSetting
|
||||
{
|
||||
extern const TimeSeriesSettingsBool aggregate_min_time_and_max_time;
|
||||
extern const TimeSeriesSettingsBool store_min_time_and_max_time;
|
||||
extern const TimeSeriesSettingsMap tags_to_columns;
|
||||
extern const TimeSeriesSettingsBool use_all_tags_column_to_generate_id;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCOMPATIBLE_COLUMNS;
|
||||
@ -85,7 +93,7 @@ void TimeSeriesDefinitionNormalizer::reorderColumns(ASTCreateQuery & create) con
|
||||
/// Reorder columns for the "tags" table.
|
||||
add_column_in_correct_order(TimeSeriesColumnNames::MetricName);
|
||||
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -96,7 +104,7 @@ void TimeSeriesDefinitionNormalizer::reorderColumns(ASTCreateQuery & create) con
|
||||
add_column_in_correct_order(TimeSeriesColumnNames::Tags);
|
||||
add_column_in_correct_order(TimeSeriesColumnNames::AllTags);
|
||||
|
||||
if (time_series_settings.store_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::store_min_time_and_max_time])
|
||||
{
|
||||
add_column_in_correct_order(TimeSeriesColumnNames::MinTime);
|
||||
add_column_in_correct_order(TimeSeriesColumnNames::MaxTime);
|
||||
@ -198,7 +206,7 @@ void TimeSeriesDefinitionNormalizer::addMissingColumns(ASTCreateQuery & create)
|
||||
make_new_column(TimeSeriesColumnNames::MetricName, get_lc_string_type());
|
||||
}
|
||||
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -221,7 +229,7 @@ void TimeSeriesDefinitionNormalizer::addMissingColumns(ASTCreateQuery & create)
|
||||
make_new_column(TimeSeriesColumnNames::AllTags, get_string_to_string_map_type());
|
||||
}
|
||||
|
||||
if (time_series_settings.store_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::store_min_time_and_max_time])
|
||||
{
|
||||
/// We use Nullable(DateTime64(3)) as the default type of the `min_time` and `max_time` columns.
|
||||
/// It's nullable because it allows the aggregation (see aggregate_min_time_and_max_time) work correctly even
|
||||
@ -284,13 +292,13 @@ ASTPtr TimeSeriesDefinitionNormalizer::chooseIDAlgorithm(const ASTColumnDeclarat
|
||||
ASTs arguments_for_hash_function;
|
||||
arguments_for_hash_function.push_back(std::make_shared<ASTIdentifier>(TimeSeriesColumnNames::MetricName));
|
||||
|
||||
if (time_series_settings.use_all_tags_column_to_generate_id)
|
||||
if (time_series_settings[TimeSeriesSetting::use_all_tags_column_to_generate_id])
|
||||
{
|
||||
arguments_for_hash_function.push_back(std::make_shared<ASTIdentifier>(TimeSeriesColumnNames::AllTags));
|
||||
}
|
||||
else
|
||||
{
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -424,7 +432,7 @@ void TimeSeriesDefinitionNormalizer::setInnerEngineByDefault(ViewTarget::Kind in
|
||||
case ViewTarget::Tags:
|
||||
{
|
||||
String engine_name;
|
||||
if (time_series_settings.aggregate_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::aggregate_min_time_and_max_time])
|
||||
engine_name = "AggregatingMergeTree";
|
||||
else
|
||||
engine_name = "ReplacingMergeTree";
|
||||
@ -441,7 +449,7 @@ void TimeSeriesDefinitionNormalizer::setInnerEngineByDefault(ViewTarget::Kind in
|
||||
order_by_list.push_back(std::make_shared<ASTIdentifier>(TimeSeriesColumnNames::MetricName));
|
||||
order_by_list.push_back(std::make_shared<ASTIdentifier>(TimeSeriesColumnNames::ID));
|
||||
|
||||
if (time_series_settings.store_min_time_and_max_time && !time_series_settings.aggregate_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::store_min_time_and_max_time] && !time_series_settings[TimeSeriesSetting::aggregate_min_time_and_max_time])
|
||||
{
|
||||
order_by_list.push_back(std::make_shared<ASTIdentifier>(TimeSeriesColumnNames::MinTime));
|
||||
order_by_list.push_back(std::make_shared<ASTIdentifier>(TimeSeriesColumnNames::MaxTime));
|
||||
|
@ -22,6 +22,14 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace TimeSeriesSetting
|
||||
{
|
||||
extern const TimeSeriesSettingsBool aggregate_min_time_and_max_time;
|
||||
extern const TimeSeriesSettingsBool store_min_time_and_max_time;
|
||||
extern const TimeSeriesSettingsMap tags_to_columns;
|
||||
extern const TimeSeriesSettingsBool use_all_tags_column_to_generate_id;
|
||||
}
|
||||
|
||||
TimeSeriesInnerTablesCreator::TimeSeriesInnerTablesCreator(ContextPtr context_,
|
||||
StorageID time_series_storage_id_,
|
||||
std::reference_wrapper<const ColumnsDescription> time_series_columns_,
|
||||
@ -70,7 +78,7 @@ ColumnsDescription TimeSeriesInnerTablesCreator::getInnerTableColumnsDescription
|
||||
columns.add(time_series_columns.get(TimeSeriesColumnNames::MetricName));
|
||||
|
||||
/// Columns corresponding to specific tags specified in the "tags_to_columns" setting.
|
||||
const Map & tags_to_columns = time_series_settings.tags_to_columns;
|
||||
const Map & tags_to_columns = time_series_settings[TimeSeriesSetting::tags_to_columns];
|
||||
for (const auto & tag_name_and_column_name : tags_to_columns)
|
||||
{
|
||||
const auto & tuple = tag_name_and_column_name.safeGet<const Tuple &>();
|
||||
@ -82,7 +90,7 @@ ColumnsDescription TimeSeriesInnerTablesCreator::getInnerTableColumnsDescription
|
||||
columns.add(time_series_columns.get(TimeSeriesColumnNames::Tags));
|
||||
|
||||
/// Column "all_tags".
|
||||
if (time_series_settings.use_all_tags_column_to_generate_id)
|
||||
if (time_series_settings[TimeSeriesSetting::use_all_tags_column_to_generate_id])
|
||||
{
|
||||
ColumnDescription all_tags_column = time_series_columns.get(TimeSeriesColumnNames::AllTags);
|
||||
/// Column "all_tags" is here only to calculate the identifier of a time series for the "id" column, so it can be ephemeral.
|
||||
@ -96,11 +104,11 @@ ColumnsDescription TimeSeriesInnerTablesCreator::getInnerTableColumnsDescription
|
||||
}
|
||||
|
||||
/// Columns "min_time" and "max_time".
|
||||
if (time_series_settings.store_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::store_min_time_and_max_time])
|
||||
{
|
||||
auto min_time_column = time_series_columns.get(TimeSeriesColumnNames::MinTime);
|
||||
auto max_time_column = time_series_columns.get(TimeSeriesColumnNames::MaxTime);
|
||||
if (time_series_settings.aggregate_min_time_and_max_time)
|
||||
if (time_series_settings[TimeSeriesSetting::aggregate_min_time_and_max_time])
|
||||
{
|
||||
AggregateFunctionProperties properties;
|
||||
auto min_function = AggregateFunctionFactory::instance().get("min", NullsAction::EMPTY, {min_time_column.type}, {}, properties);
|
||||
|
@ -1,7 +1,8 @@
|
||||
#include <Storages/TimeSeries/TimeSeriesSettings.h>
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacrosImpl.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Storages/TimeSeries/TimeSeriesSettings.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -12,15 +13,53 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_SETTING;
|
||||
}
|
||||
|
||||
#define LIST_OF_TIME_SERIES_SETTINGS(M, ALIAS) \
|
||||
M(Map, tags_to_columns, Map{}, "Map specifying which tags should be put to separate columns of the 'tags' table. Syntax: {'tag1': 'column1', 'tag2' : column2, ...}", 0) \
|
||||
M(Bool, use_all_tags_column_to_generate_id, true, "When generating an expression to calculate an identifier of a time series, this flag enables using the 'all_tags' column in that calculation. The 'all_tags' is a virtual column containing all tags except the metric name", 0) \
|
||||
M(Bool, store_min_time_and_max_time, true, "If set to true then the table will store 'min_time' and 'max_time' for each time series", 0) \
|
||||
M(Bool, aggregate_min_time_and_max_time, true, "When creating an inner target 'tags' table, this flag enables using 'SimpleAggregateFunction(min, Nullable(DateTime64(3)))' instead of just 'Nullable(DateTime64(3))' as the type of the 'min_time' column, and the same for the 'max_time' column", 0) \
|
||||
M(Bool, filter_by_min_time_and_max_time, true, "If set to true then the table will use the 'min_time' and 'max_time' columns for filtering time series", 0) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(TimeSeriesSettingsTraits, LIST_OF_TIME_SERIES_SETTINGS)
|
||||
IMPLEMENT_SETTINGS_TRAITS(TimeSeriesSettingsTraits, LIST_OF_TIME_SERIES_SETTINGS)
|
||||
|
||||
struct TimeSeriesSettingsImpl : public BaseSettings<TimeSeriesSettingsTraits>
|
||||
{
|
||||
};
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) TimeSeriesSettings##TYPE NAME = &TimeSeriesSettingsImpl ::NAME;
|
||||
|
||||
namespace TimeSeriesSetting
|
||||
{
|
||||
LIST_OF_TIME_SERIES_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
}
|
||||
|
||||
#undef INITIALIZE_SETTING_EXTERN
|
||||
|
||||
TimeSeriesSettings::TimeSeriesSettings() : impl(std::make_unique<TimeSeriesSettingsImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
TimeSeriesSettings::TimeSeriesSettings(const TimeSeriesSettings & settings) : impl(std::make_unique<TimeSeriesSettingsImpl>(*settings.impl))
|
||||
{
|
||||
}
|
||||
|
||||
TimeSeriesSettings::TimeSeriesSettings(TimeSeriesSettings && settings) noexcept
|
||||
: impl(std::make_unique<TimeSeriesSettingsImpl>(std::move(*settings.impl)))
|
||||
{
|
||||
}
|
||||
|
||||
TimeSeriesSettings::~TimeSeriesSettings() = default;
|
||||
|
||||
TIMESERIES_SETTINGS_SUPPORTED_TYPES(TimeSeriesSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void TimeSeriesSettings::loadFromQuery(ASTStorage & storage_def)
|
||||
{
|
||||
if (storage_def.settings)
|
||||
{
|
||||
try
|
||||
{
|
||||
applyChanges(storage_def.settings->changes);
|
||||
impl->applyChanges(storage_def.settings->changes);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
|
@ -1,29 +1,36 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTStorage;
|
||||
struct TimeSeriesSettingsImpl;
|
||||
|
||||
#define LIST_OF_TIME_SERIES_SETTINGS(M, ALIAS) \
|
||||
M(Map, tags_to_columns, Map{}, "Map specifying which tags should be put to separate columns of the 'tags' table. Syntax: {'tag1': 'column1', 'tag2' : column2, ...}", 0) \
|
||||
M(Bool, use_all_tags_column_to_generate_id, true, "When generating an expression to calculate an identifier of a time series, this flag enables using the 'all_tags' column in that calculation. The 'all_tags' is a virtual column containing all tags except the metric name", 0) \
|
||||
M(Bool, store_min_time_and_max_time, true, "If set to true then the table will store 'min_time' and 'max_time' for each time series", 0) \
|
||||
M(Bool, aggregate_min_time_and_max_time, true, "When creating an inner target 'tags' table, this flag enables using 'SimpleAggregateFunction(min, Nullable(DateTime64(3)))' instead of just 'Nullable(DateTime64(3))' as the type of the 'min_time' column, and the same for the 'max_time' column", 0) \
|
||||
M(Bool, filter_by_min_time_and_max_time, true, "If set to true then the table will use the 'min_time' and 'max_time' columns for filtering time series", 0) \
|
||||
/// List of available types supported in TimeSeriesSettings object
|
||||
#define TIMESERIES_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, Map)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(TimeSeriesSettingsTraits, LIST_OF_TIME_SERIES_SETTINGS)
|
||||
TIMESERIES_SETTINGS_SUPPORTED_TYPES(TimeSeriesSettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
/// Settings for the TimeSeries table engine.
|
||||
/// Could be loaded from a CREATE TABLE query (SETTINGS clause). For example:
|
||||
/// CREATE TABLE mytable ENGINE = TimeSeries() SETTINGS tags_to_columns = {'job':'job', 'instance':'instance'} DATA ENGINE = ReplicatedMergeTree('zkpath', 'replica'), ...
|
||||
struct TimeSeriesSettings : public BaseSettings<TimeSeriesSettingsTraits>
|
||||
struct TimeSeriesSettings
|
||||
{
|
||||
TimeSeriesSettings();
|
||||
TimeSeriesSettings(const TimeSeriesSettings & settings);
|
||||
TimeSeriesSettings(TimeSeriesSettings && settings) noexcept;
|
||||
~TimeSeriesSettings();
|
||||
|
||||
TIMESERIES_SETTINGS_SUPPORTED_TYPES(TimeSeriesSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void loadFromQuery(ASTStorage & storage_def);
|
||||
|
||||
private:
|
||||
std::unique_ptr<TimeSeriesSettingsImpl> impl;
|
||||
};
|
||||
|
||||
using TimeSeriesSettingsPtr = std::shared_ptr<const TimeSeriesSettings>;
|
||||
|
||||
}
|
||||
|
@ -32,6 +32,7 @@ def request(command, url, headers={}, data=None):
|
||||
|
||||
|
||||
class RequestHandler(http.server.BaseHTTPRequestHandler):
|
||||
# GetObject
|
||||
def do_GET(self):
|
||||
if self.path == "/":
|
||||
self.send_response(200)
|
||||
@ -41,12 +42,18 @@ class RequestHandler(http.server.BaseHTTPRequestHandler):
|
||||
else:
|
||||
self.do_HEAD()
|
||||
|
||||
# PutObject
|
||||
def do_PUT(self):
|
||||
self.do_HEAD()
|
||||
|
||||
# DeleteObjects (/root?delete)
|
||||
def do_POST(self):
|
||||
self.do_HEAD()
|
||||
|
||||
# DeleteObject
|
||||
def do_DELETE(self):
|
||||
self.do_HEAD()
|
||||
|
||||
def do_HEAD(self):
|
||||
content_length = self.headers.get("Content-Length")
|
||||
data = self.rfile.read(int(content_length)) if content_length else None
|
||||
|
@ -16,6 +16,14 @@ CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similar
|
||||
-- deterministic randomness.
|
||||
INSERT INTO tab SELECT number, [sipHash64(number)/18446744073709551615, wyHash64(number)/18446744073709551615] FROM numbers(370000); -- 18446744073709551615 is the biggest UInt64
|
||||
|
||||
-- hnsw_candidate_list_size_for_search = 0 is illegal
|
||||
WITH [0.5, 0.5] AS reference_vec
|
||||
SELECT id, vec, L2Distance(vec, reference_vec)
|
||||
FROM tab
|
||||
ORDER BY L2Distance(vec, reference_vec)
|
||||
LIMIT 3
|
||||
SETTINGS hnsw_candidate_list_size_for_search = 0; -- { serverError INVALID_SETTING_VALUE }
|
||||
|
||||
DROP TABLE IF EXISTS results;
|
||||
CREATE TABLE results(id Int32) ENGINE = Memory;
|
||||
|
||||
|
@ -64,3 +64,4 @@ SELECT * FROM t1n as t1 JOIN t2n as t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL)
|
||||
0
|
||||
2
|
||||
2
|
||||
1
|
||||
|
@ -67,6 +67,10 @@ SELECT count() FROM ( EXPLAIN QUERY TREE
|
||||
SELECT * FROM t1 JOIN t2 ON t1.x <=> t2.x AND (t1.x = t1.y OR t1.x IS NULL AND t1.y IS NULL)
|
||||
) WHERE explain like '%CONSTANT%' OR explain ilike '%is%null%';
|
||||
|
||||
SELECT count() FROM ( EXPLAIN QUERY TREE
|
||||
SELECT * FROM t1 JOIN t2 ON t1.x = t2.x AND NOT (t1.x = 1 OR t1.x IS NULL)
|
||||
) WHERE explain ilike '%function_name: isNull%';
|
||||
|
||||
DROP TABLE IF EXISTS t1;
|
||||
DROP TABLE IF EXISTS t2;
|
||||
DROP TABLE IF EXISTS t1n;
|
||||
|
@ -0,0 +1,38 @@
|
||||
--
|
||||
0
|
||||
SELECT NOT 1
|
||||
SELECT NOT 1
|
||||
--
|
||||
SELECT not(1, 1, 1)
|
||||
SELECT not(1, 1, 1)
|
||||
--
|
||||
SELECT NOT tuple(1)
|
||||
SELECT NOT tuple(1)
|
||||
SELECT NOT tuple(1)
|
||||
--
|
||||
SELECT NOT ((1, 1, 1))
|
||||
SELECT NOT ((1, 1, 1))
|
||||
SELECT NOT ((1, 1, 1))
|
||||
SELECT NOT ((1, 1, 1))
|
||||
--
|
||||
SELECT NOT [1]
|
||||
SELECT NOT [1]
|
||||
SELECT NOT [1]
|
||||
SELECT NOT [1]
|
||||
SELECT NOT [1]
|
||||
--
|
||||
SELECT -[1]
|
||||
SELECT -[1]
|
||||
SELECT -[1]
|
||||
SELECT -[1]
|
||||
SELECT -[1]
|
||||
--
|
||||
(-1,-1,-1)
|
||||
SELECT -(1, 1, 1)
|
||||
SELECT -(1, 1, 1)
|
||||
SELECT -(1, 1, 1)
|
||||
SELECT -(1, 1, 1)
|
||||
--
|
||||
((-1,-1,-1))
|
||||
SELECT -tuple((1, 1, 1))
|
||||
SELECT -tuple((1, 1, 1))
|
@ -5,3 +5,57 @@ create table a (`value2` Enum8('Hello' = 1, equals(`Null`, 'World', 2), '!' = 3)
|
||||
|
||||
create table a (x Int8) engine Memory;
|
||||
create table b empty as a;
|
||||
|
||||
SELECT '--';
|
||||
SELECT NOT (1);
|
||||
SELECT formatQuery('SELECT NOT 1');
|
||||
SELECT formatQuery('SELECT NOT (1)');
|
||||
|
||||
SELECT '--';
|
||||
SELECT NOT (1, 1, 1); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
SELECT formatQuery('SELECT NOT (1, 1, 1)');
|
||||
SELECT formatQuery('SELECT not(1, 1, 1)');
|
||||
|
||||
SELECT '--';
|
||||
SELECT NOT ((1,)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT NOT tuple(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT formatQuery('SELECT NOT ((1,))');
|
||||
SELECT formatQuery('SELECT NOT (tuple(1))');
|
||||
SELECT formatQuery('SELECT NOT tuple(1)');
|
||||
|
||||
SELECT '--';
|
||||
SELECT NOT ((1, 1, 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT formatQuery('SELECT NOT ((1, 1, 1))');
|
||||
SELECT formatQuery('SELECT not((1, 1, 1))');
|
||||
SELECT formatQuery('SELECT not tuple(1, 1, 1)');
|
||||
SELECT formatQuery('SELECT not (tuple(1, 1, 1))');
|
||||
|
||||
SELECT '--';
|
||||
SELECT NOT [1]; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT NOT [(1)]; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT formatQuery('SELECT NOT [1]');
|
||||
SELECT formatQuery('SELECT NOT array(1)');
|
||||
SELECT formatQuery('SELECT NOT (array(1))');
|
||||
SELECT formatQuery('SELECT NOT [(1)]');
|
||||
SELECT formatQuery('SELECT NOT ([1])');
|
||||
|
||||
SELECT '--';
|
||||
SELECT -[1]; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT -[(1)]; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT formatQuery('SELECT -[1]');
|
||||
SELECT formatQuery('SELECT -array(1)');
|
||||
SELECT formatQuery('SELECT -(array(1))');
|
||||
SELECT formatQuery('SELECT -[(1)]');
|
||||
SELECT formatQuery('SELECT -([1])');
|
||||
|
||||
SELECT '--';
|
||||
SELECT -(1, 1, 1);
|
||||
SELECT formatQuery('SELECT -(1, 1, 1)');
|
||||
SELECT formatQuery('SELECT negate ((1, 1, 1))');
|
||||
SELECT formatQuery('SELECT -tuple(1, 1, 1)');
|
||||
SELECT formatQuery('SELECT -(tuple(1, 1, 1))');
|
||||
|
||||
SELECT '--';
|
||||
SELECT -tuple((1, 1, 1));
|
||||
SELECT formatQuery('SELECT -((1, 1, 1))');
|
||||
SELECT formatQuery('SELECT -tuple((1, 1, 1))');
|
||||
|
115
utils/check-style/check-settings-style
Executable file
115
utils/check-style/check-settings-style
Executable file
@ -0,0 +1,115 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
# Fast check of all the setting struct usages
|
||||
# The linker does not complain about incorrect extern usage, so we need to make sure the style checker handles
|
||||
|
||||
LC_ALL="en_US.UTF-8"
|
||||
ROOT_PATH=$(git rev-parse --show-toplevel)
|
||||
|
||||
# Duplicated or incorrect setting declarations
|
||||
SETTINGS_FILE=$(mktemp)
|
||||
ALL_DECLARATION_FILES="
|
||||
$ROOT_PATH/src/Core/Settings.cpp
|
||||
$ROOT_PATH/src/Core/ServerSettings.cpp
|
||||
$ROOT_PATH/src/Storages/MergeTree/MergeTreeSettings.cpp
|
||||
$ROOT_PATH/src/Coordination/CoordinationSettings.cpp
|
||||
$ROOT_PATH/src/Databases/DatabaseReplicatedSettings.cpp
|
||||
$ROOT_PATH/src/Storages/TimeSeries/TimeSeriesSettings.cpp
|
||||
$ROOT_PATH/src/Storages/RocksDB/RocksDBSettings.cpp
|
||||
$ROOT_PATH/src/Storages/RabbitMQ/RabbitMQSettings.cpp
|
||||
$ROOT_PATH/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.cpp
|
||||
$ROOT_PATH/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.cpp
|
||||
$ROOT_PATH/src/Storages/MaterializedView/RefreshSettings.cpp
|
||||
$ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h"
|
||||
|
||||
for settings_file in ${ALL_DECLARATION_FILES};
|
||||
do
|
||||
if ! [ -f "${settings_file}" ]; then
|
||||
echo "File '${settings_file}' does not exist."
|
||||
fi
|
||||
done
|
||||
|
||||
cat $ROOT_PATH/src/Core/Settings.cpp $ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " Settings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq > ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Core/ServerSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " ServerSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/MergeTree/MergeTreeSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " MergeTreeSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Coordination/CoordinationSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " CoordinationSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Databases/DatabaseReplicatedSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " DatabaseReplicatedSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/TimeSeries/TimeSeriesSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " TimeSeriesSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/RocksDB/RocksDBSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " RocksDBSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/RabbitMQ/RabbitMQSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " RabbitMQSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " MaterializedPostgreSQLSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " ObjectStorageQueueSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/MaterializedView/RefreshSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " RefreshSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
|
||||
|
||||
# Check that if there are duplicated settings (declared in different objects) they all have the same type (it's simpler to validate style with that assert)
|
||||
for setting in $(awk '{print $1 " " $2}' ${SETTINGS_FILE} | \
|
||||
sed -e 's/CoordinationSettings//g' \
|
||||
-e 's/DatabaseReplicatedSettings//g' \
|
||||
-e 's/TimeSeriesSettings//g' \
|
||||
-e 's/RabbitMQSettings//g' \
|
||||
-e 's/RocksDBSettings//g' \
|
||||
-e 's/MaterializedPostgreSQLSettings//g' \
|
||||
-e 's/ObjectStorageQueueSettings//g' \
|
||||
-e 's/RefreshSettings//g' \
|
||||
-e 's/MergeTreeSettings//g' \
|
||||
-e 's/ServerSettings//g' \
|
||||
-e 's/Settings//g' | \
|
||||
sort | uniq | awk '{ print $1 }' | uniq -d);
|
||||
do
|
||||
echo "# Found multiple definitions of setting ${setting} with different types: "
|
||||
grep --line-number " ${setting}," ${ALL_DECLARATION_FILES} | awk '{print " > " $0 }'
|
||||
done
|
||||
|
||||
# We append all uses of extern found in implementation files to validate them in a single pass and avoid reading the same files over and over
|
||||
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | \
|
||||
xargs grep -e "^\s*extern const Settings" \
|
||||
-e "^\s**extern const ServerSettings" \
|
||||
-e "^\s**extern const MergeTreeSettings" \
|
||||
-e "^\s**extern const RabbitMQSettings" \
|
||||
-e "^\s**extern const RocksDBSettings" \
|
||||
-e "^\s**extern const MaterializedPostgreSQLSettings" \
|
||||
-e "^\s**extern const ObjectStorageQueueSettings" \
|
||||
-e "^\s**extern const RefreshSettings" \
|
||||
-e "^\s**extern const TimeSeriesSettings" \
|
||||
-e "^\s**extern const DatabaseReplicatedSettings" \
|
||||
-e "^\s**extern const CoordinationSettings" -T | \
|
||||
awk '{print substr($5, 0, length($5) -1) " " $4 " " substr($1, 0, length($1) - 1)}' >> ${SETTINGS_FILE}
|
||||
|
||||
# Duplicate extern declarations for settings
|
||||
awk '{if (seen[$0]++) print $3 " -> " $1 ;}' ${SETTINGS_FILE} | while read line;
|
||||
do
|
||||
echo "# Found duplicated setting declaration in: $line"
|
||||
done
|
||||
|
||||
# Find missing declarations (obsolete settings being used)
|
||||
# Note that SettingsDeclaration are first in the file
|
||||
# Disabled for now pending fixing the code
|
||||
#awk '{print $1 " " $3}' ${SETTINGS_FILE} | awk '{if (!seen[$1]++) print $0}' | grep -v SettingsDeclaration | while read setting;
|
||||
#do
|
||||
# echo "Could not find setting (maybe obsolete but used?) $setting"
|
||||
#done
|
||||
|
||||
# Look for settings declared with multiple types
|
||||
for setting in $(awk '{print $1 " " $2}' ${SETTINGS_FILE} | \
|
||||
sed -e 's/MergeTreeSettings//g' \
|
||||
-e 's/ServerSettings//g' \
|
||||
-e 's/CoordinationSettings//g' \
|
||||
-e 's/TimeSeriesSettings//g' \
|
||||
-e 's/RabbitMQSettings//g' \
|
||||
-e 's/RefreshSettings//g' \
|
||||
-e 's/RocksDBSettings//g' \
|
||||
-e 's/MaterializedPostgreSQLSettings//g' \
|
||||
-e 's/ObjectStorageQueueSettings//g' \
|
||||
-e 's/DatabaseReplicatedSettings//g' \
|
||||
-e 's/Settings//g' | \
|
||||
sort | uniq | awk '{ print $1 }' | sort | uniq -d);
|
||||
do
|
||||
expected=$(grep "^$setting " ${SETTINGS_FILE} | grep SettingsDeclaration | awk '{ print $2 }')
|
||||
grep "^$setting " ${SETTINGS_FILE} | grep -v " $expected" | awk '{ print $3 " found setting " $1 " with type " $2 }' | while read line;
|
||||
do
|
||||
echo "# In $line but it should be ${expected/$'\n'/ }"
|
||||
done
|
||||
done
|
||||
|
||||
rm ${SETTINGS_FILE}
|
@ -53,69 +53,7 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' 2>/dev/n
|
||||
find -L $ROOT_PATH -type l 2>/dev/null | grep -v contrib && echo "^ Broken symlinks found"
|
||||
|
||||
# Duplicated or incorrect setting declarations
|
||||
SETTINGS_FILE=$(mktemp)
|
||||
ALL_DECLARATION_FILES="
|
||||
$ROOT_PATH/src/Core/Settings.cpp
|
||||
$ROOT_PATH/src/Core/ServerSettings.cpp
|
||||
$ROOT_PATH/src/Storages/MergeTree/MergeTreeSettings.cpp
|
||||
$ROOT_PATH/src/Coordination/CoordinationSettings.cpp
|
||||
$ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h"
|
||||
|
||||
cat $ROOT_PATH/src/Core/Settings.cpp $ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " Settings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq > ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Core/ServerSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " ServerSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/MergeTree/MergeTreeSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " MergeTreeSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Coordination/CoordinationSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " CoordinationSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
|
||||
# Check that if there are duplicated settings (declared in different objects) they all have the same type (it's simpler to validate style with that assert)
|
||||
for setting in $(awk '{print $1 " " $2}' ${SETTINGS_FILE} | \
|
||||
sed -e 's/CoordinationSettings//g' \
|
||||
-e 's/MergeTreeSettings//g' \
|
||||
-e 's/ServerSettings//g' \
|
||||
-e 's/Settings//g' | \
|
||||
sort | uniq | awk '{ print $1 }' | uniq -d);
|
||||
do
|
||||
echo "# Found multiple definitions of setting ${setting} with different types: "
|
||||
grep --line-number " ${setting}," ${ALL_DECLARATION_FILES} | awk '{print " > " $0 }'
|
||||
done
|
||||
|
||||
# We append all uses of extern found in implementation files to validate them in a single pass and avoid reading the same files over and over
|
||||
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | \
|
||||
xargs grep -e "^\s*extern const Settings" \
|
||||
-e "^\s**extern const ServerSettings" \
|
||||
-e "^\s**extern const MergeTreeSettings" \
|
||||
-e "^\s**extern const CoordinationSettings" -T | \
|
||||
awk '{print substr($5, 0, length($5) -1) " " $4 " " substr($1, 0, length($1) - 1)}' >> ${SETTINGS_FILE}
|
||||
|
||||
# Duplicate extern declarations for settings
|
||||
awk '{if (seen[$0]++) print $3 " -> " $1 ;}' ${SETTINGS_FILE} | while read line;
|
||||
do
|
||||
echo "# Found duplicated setting declaration in: $line"
|
||||
done
|
||||
|
||||
# Find missing declarations (obsolete settings being used)
|
||||
# Note that SettingsDeclaration are first in the file
|
||||
# Disabled for now pending fixing the code
|
||||
#awk '{print $1 " " $3}' ${SETTINGS_FILE} | awk '{if (!seen[$1]++) print $0}' | grep -v SettingsDeclaration | while read setting;
|
||||
#do
|
||||
# echo "Could not find setting (maybe obsolete but used?) $setting"
|
||||
#done
|
||||
|
||||
# Look for settings declared with multiple types
|
||||
for setting in $(awk '{print $1 " " $2}' ${SETTINGS_FILE} | \
|
||||
sed -e 's/MergeTreeSettings//g' \
|
||||
-e 's/ServerSettings//g' \
|
||||
-e 's/CoordinationSettings//g' \
|
||||
-e 's/Settings//g' | \
|
||||
sort | uniq | awk '{ print $1 }' | sort | uniq -d);
|
||||
do
|
||||
expected=$(grep "^$setting " ${SETTINGS_FILE} | grep SettingsDeclaration | awk '{ print $2 }')
|
||||
grep "^$setting " ${SETTINGS_FILE} | grep -v " $expected" | awk '{ print $3 " found setting " $1 " with type " $2 }' | while read line;
|
||||
do
|
||||
echo "# In $line but it should be ${expected/$'\n'/ }"
|
||||
done
|
||||
done
|
||||
|
||||
rm ${SETTINGS_FILE}
|
||||
bash $ROOT_PATH/utils/check-style/check-settings-style
|
||||
|
||||
# Unused/Undefined/Duplicates ErrorCodes/ProfileEvents/CurrentMetrics
|
||||
declare -A EXTERN_TYPES
|
||||
|
Loading…
Reference in New Issue
Block a user