Merge remote-tracking branch 'upstream/master' into allow-single-disk-instead-of-storage-policy

This commit is contained in:
kssenii 2022-11-22 13:58:23 +01:00
commit 5f649baffe
120 changed files with 3027 additions and 706 deletions

File diff suppressed because it is too large Load Diff

View File

@ -254,7 +254,7 @@ sudo chgrp clickhouse /etc/clickhouse-server/config.d/s3_storage_policy_by_defau
start start
./stress --hung-check --drop-databases --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" \ ./stress --hung-check --drop-databases --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" --global-time-limit 1200 \
&& echo -e 'Test script exit code\tOK' >> /test_output/test_results.tsv \ && echo -e 'Test script exit code\tOK' >> /test_output/test_results.tsv \
|| echo -e 'Test script failed\tFAIL' >> /test_output/test_results.tsv || echo -e 'Test script failed\tFAIL' >> /test_output/test_results.tsv

View File

@ -0,0 +1,31 @@
---
sidebar_position: 1
sidebar_label: 2022
---
# 2022 Changelog
### ClickHouse release v22.8.9.24-lts (a1b69551d40) FIXME as compared to v22.8.8.3-lts (ac5a6cababc)
#### Performance Improvement
* Backported in [#43012](https://github.com/ClickHouse/ClickHouse/issues/43012): Keeper performance improvement: improve commit performance for cases when many different nodes have uncommitted states. This should help with cases when a follower node can't sync fast enough. [#42926](https://github.com/ClickHouse/ClickHouse/pull/42926) ([Antonio Andelic](https://github.com/antonio2368)).
#### Improvement
* Backported in [#42840](https://github.com/ClickHouse/ClickHouse/issues/42840): Update tzdata to 2022f. Mexico will no longer observe DST except near the US border: https://www.timeanddate.com/news/time/mexico-abolishes-dst-2022.html. Chihuahua moves to year-round UTC-6 on 2022-10-30. Fiji no longer observes DST. See https://github.com/google/cctz/pull/235 and https://bugs.launchpad.net/ubuntu/+source/tzdata/+bug/1995209. [#42796](https://github.com/ClickHouse/ClickHouse/pull/42796) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
#### Build/Testing/Packaging Improvement
* Backported in [#42964](https://github.com/ClickHouse/ClickHouse/issues/42964): Before the fix, the user-defined config was preserved by RPM in `$file.rpmsave`. The PR fixes it and won't replace the user's files from packages. [#42936](https://github.com/ClickHouse/ClickHouse/pull/42936) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Backported in [#43040](https://github.com/ClickHouse/ClickHouse/issues/43040): Add a CI step to mark commits as ready for release; soft-forbid launching a release script from branches but master. [#43017](https://github.com/ClickHouse/ClickHouse/pull/43017) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
#### Bug Fix (user-visible misbehavior in official stable or prestable release)
* Backported in [#42720](https://github.com/ClickHouse/ClickHouse/issues/42720): Fixed `Unknown identifier (aggregate-function)` exception which appears when a user tries to calculate WINDOW ORDER BY/PARTITION BY expressions over aggregate functions: ``` CREATE TABLE default.tenk1 ( `unique1` Int32, `unique2` Int32, `ten` Int32 ) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192; SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM _complex GROUP BY ten ORDER BY ten ASC; ``` which gives: ``` Code: 47. DB::Exception: Received from localhost:9000. DB::Exception: Unknown identifier: sum(unique1); there are columns: unique1, unique2, ten: While processing sum(unique1) + sum(unique2) ASC. (UNKNOWN_IDENTIFIER) ```. [#39762](https://github.com/ClickHouse/ClickHouse/pull/39762) ([Vladimir Chebotaryov](https://github.com/quickhouse)).
* Backported in [#42748](https://github.com/ClickHouse/ClickHouse/issues/42748): A segmentation fault related to DNS & c-ares has been reported. The below error ocurred in multiple threads: ``` 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008088 [ 356 ] {} <Fatal> BaseDaemon: ######################################## 2022-09-28 15:41:19.008,"2022.09.28 15:41:19.008147 [ 356 ] {} <Fatal> BaseDaemon: (version 22.8.5.29 (official build), build id: 92504ACA0B8E2267) (from thread 353) (no query) Received signal Segmentation fault (11)" 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008196 [ 356 ] {} <Fatal> BaseDaemon: Address: 0xf Access: write. Address not mapped to object. 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008216 [ 356 ] {} <Fatal> BaseDaemon: Stack trace: 0x188f8212 0x1626851b 0x1626a69e 0x16269b3f 0x16267eab 0x13cf8284 0x13d24afc 0x13c5217e 0x14ec2495 0x15ba440f 0x15b9d13b 0x15bb2699 0x1891ccb3 0x1891e00d 0x18ae0769 0x18ade022 0x7f76aa985609 0x7f76aa8aa133 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008274 [ 356 ] {} <Fatal> BaseDaemon: 2. Poco::Net::IPAddress::family() const @ 0x188f8212 in /usr/bin/clickhouse 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008297 [ 356 ] {} <Fatal> BaseDaemon: 3. ? @ 0x1626851b in /usr/bin/clickhouse 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008309 [ 356 ] {} <Fatal> BaseDaemon: 4. ? @ 0x1626a69e in /usr/bin/clickhouse ```. [#42234](https://github.com/ClickHouse/ClickHouse/pull/42234) ([Arthur Passos](https://github.com/arthurpassos)).
* Backported in [#43062](https://github.com/ClickHouse/ClickHouse/issues/43062): Fix rare NOT_FOUND_COLUMN_IN_BLOCK error when projection is possible to use but there is no projection available. This fixes [#42771](https://github.com/ClickHouse/ClickHouse/issues/42771) . The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/25563. [#42938](https://github.com/ClickHouse/ClickHouse/pull/42938) ([Amos Bird](https://github.com/amosbird)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Do not warn about kvm-clock [#41217](https://github.com/ClickHouse/ClickHouse/pull/41217) ([Sergei Trifonov](https://github.com/serxa)).
* Revert revert 41268 disable s3 parallel write for part moves to disk s3 [#42617](https://github.com/ClickHouse/ClickHouse/pull/42617) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Always run `BuilderReport` and `BuilderSpecialReport` in all CI types [#42684](https://github.com/ClickHouse/ClickHouse/pull/42684) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).

View File

@ -57,7 +57,7 @@ Internal coordination settings are located in the `<keeper_server>.<coordination
- `auto_forwarding` — Allow to forward write requests from followers to the leader (default: true). - `auto_forwarding` — Allow to forward write requests from followers to the leader (default: true).
- `shutdown_timeout` — Wait to finish internal connections and shutdown (ms) (default: 5000). - `shutdown_timeout` — Wait to finish internal connections and shutdown (ms) (default: 5000).
- `startup_timeout` — If the server doesn't connect to other quorum participants in the specified timeout it will terminate (ms) (default: 30000). - `startup_timeout` — If the server doesn't connect to other quorum participants in the specified timeout it will terminate (ms) (default: 30000).
- `four_letter_word_white_list` — White list of 4lw commands (default: `conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro`). - `four_letter_word_white_list` — White list of 4lw commands (default: `conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld`).
Quorum configuration is located in the `<keeper_server>.<raft_configuration>` section and contain servers description. Quorum configuration is located in the `<keeper_server>.<raft_configuration>` section and contain servers description.
@ -126,7 +126,7 @@ clickhouse keeper --config /etc/your_path_to_config/config.xml
ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively. ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively.
The 4lw commands has a white list configuration `four_letter_word_white_list` which has default value `conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif`. The 4lw commands has a white list configuration `four_letter_word_white_list` which has default value `conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld`.
You can issue the commands to ClickHouse Keeper via telnet or nc, at the client port. You can issue the commands to ClickHouse Keeper via telnet or nc, at the client port.
@ -328,6 +328,12 @@ target_committed_log_idx 101
last_snapshot_idx 50 last_snapshot_idx 50
``` ```
- `rqld`: Request to become new leader. Return `Sent leadership request to leader.` if request sent or `Failed to send leadership request to leader.` if request not sent. Note that if node is already leader the outcome is same as the request is sent.
```
Sent leadership request to leader.
```
## Migration from ZooKeeper {#migration-from-zookeeper} ## Migration from ZooKeeper {#migration-from-zookeeper}
Seamlessly migration from ZooKeeper to ClickHouse Keeper is impossible you have to stop your ZooKeeper cluster, convert data and start ClickHouse Keeper. `clickhouse-keeper-converter` tool allows converting ZooKeeper logs and snapshots to ClickHouse Keeper snapshot. It works only with ZooKeeper > 3.4. Steps for migration: Seamlessly migration from ZooKeeper to ClickHouse Keeper is impossible you have to stop your ZooKeeper cluster, convert data and start ClickHouse Keeper. `clickhouse-keeper-converter` tool allows converting ZooKeeper logs and snapshots to ClickHouse Keeper snapshot. It works only with ZooKeeper > 3.4. Steps for migration:

View File

@ -0,0 +1,75 @@
---
slug: /en/sql-reference/table-functions/format
sidebar_position: 56
sidebar_label: format
---
# format
Extracts table structure from data and parses it according to specified input format.
**Syntax**
``` sql
format(format_name, data)
```
**Parameters**
- `format_name` — The [format](../../interfaces/formats.md#formats) of the data.
- `data` — String literal or constant expression that returns a string containing data in specified format
**Returned value**
A table with data parsed from `data` argument according specified format and extracted schema.
**Examples**
**Query:**
``` sql
:) select * from format(JSONEachRow,
$$
{"a": "Hello", "b": 111}
{"a": "World", "b": 123}
{"a": "Hello", "b": 112}
{"a": "World", "b": 124}
$$)
```
**Result:**
```text
┌───b─┬─a─────┐
│ 111 │ Hello │
│ 123 │ World │
│ 112 │ Hello │
│ 124 │ World │
└─────┴───────┘
```
**Query:**
```sql
:) desc format(JSONEachRow,
$$
{"a": "Hello", "b": 111}
{"a": "World", "b": 123}
{"a": "Hello", "b": 112}
{"a": "World", "b": 124}
$$)
```
**Result:**
```text
┌─name─┬─type──────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐
│ b │ Nullable(Float64) │ │ │ │ │ │
│ a │ Nullable(String) │ │ │ │ │ │
└──────┴───────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
```
**See Also**
- [Formats](../../interfaces/formats.md)
[Original article](https://clickhouse.com/docs/en/sql-reference/table-functions/format) <!--hide-->

View File

@ -0,0 +1 @@
../../../en/sql-reference/table-functions/format.md

View File

@ -0,0 +1 @@
../../../en/sql-reference/table-functions/format.md

View File

@ -160,7 +160,7 @@ void ClusterCopierApp::mainImpl()
registerTableFunctions(); registerTableFunctions();
registerStorages(); registerStorages();
registerDictionaries(); registerDictionaries();
registerDisks(); registerDisks(/* global_skip_access_check= */ true);
registerFormats(); registerFormats();
static const std::string default_database = "_local"; static const std::string default_database = "_local";

View File

@ -176,7 +176,7 @@ int DisksApp::main(const std::vector<String> & /*args*/)
Poco::Logger::root().setLevel(Poco::Logger::parseLevel(log_level)); Poco::Logger::root().setLevel(Poco::Logger::parseLevel(log_level));
} }
registerDisks(); registerDisks(/* global_skip_access_check= */ true);
registerFormats(); registerFormats();
shared_context = Context::createShared(); shared_context = Context::createShared();

View File

@ -413,7 +413,7 @@ try
registerTableFunctions(); registerTableFunctions();
registerStorages(); registerStorages();
registerDictionaries(); registerDictionaries();
registerDisks(); registerDisks(/* global_skip_access_check= */ true);
registerFormats(); registerFormats();
processConfig(); processConfig();

View File

@ -679,7 +679,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
registerTableFunctions(); registerTableFunctions();
registerStorages(); registerStorages();
registerDictionaries(); registerDictionaries();
registerDisks(); registerDisks(/* global_skip_access_check= */ false);
registerFormats(); registerFormats();
registerRemoteFileMetadatas(); registerRemoteFileMetadatas();
@ -1148,6 +1148,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
total_memory_tracker.setDescription("(total)"); total_memory_tracker.setDescription("(total)");
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
bool allow_use_jemalloc_memory = config->getBool("allow_use_jemalloc_memory", true);
total_memory_tracker.setAllowUseJemallocMemory(allow_use_jemalloc_memory);
auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker(); auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker();
total_memory_tracker.setOvercommitTracker(global_overcommit_tracker); total_memory_tracker.setOvercommitTracker(global_overcommit_tracker);

View File

@ -13,6 +13,7 @@ struct Settings;
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int CORRUPTED_DATA;
} }
@ -89,6 +90,13 @@ public:
{ {
this->data(place).result.read(buf, *serialization_res, arena); this->data(place).result.read(buf, *serialization_res, arena);
this->data(place).value.read(buf, *serialization_val, arena); this->data(place).value.read(buf, *serialization_val, arena);
if (unlikely(this->data(place).value.has() != this->data(place).result.has()))
throw Exception(
ErrorCodes::CORRUPTED_DATA,
"Invalid state of the aggregate function {}: has_value ({}) != has_result ({})",
getName(),
this->data(place).value.has(),
this->data(place).result.has());
} }
bool allocatesMemoryInArena() const override bool allocatesMemoryInArena() const override

View File

@ -448,6 +448,34 @@ public:
}; };
struct Compatibility
{
/// Old versions used to store terminating null-character in SingleValueDataString.
/// Then -WithTerminatingZero methods were removed from IColumn interface,
/// because these methods are quite dangerous and easy to misuse. It introduced incompatibility.
/// See https://github.com/ClickHouse/ClickHouse/pull/41431 and https://github.com/ClickHouse/ClickHouse/issues/42916
/// Here we keep these functions for compatibility.
/// It's safe because there's no way unsanitized user input (without \0 at the end) can reach these functions.
static StringRef getDataAtWithTerminatingZero(const ColumnString & column, size_t n)
{
auto res = column.getDataAt(n);
/// ColumnString always reserves extra byte for null-character after string.
/// But getDataAt returns StringRef without the null-character. Let's add it.
chassert(res.data[res.size] == '\0');
++res.size;
return res;
}
static void insertDataWithTerminatingZero(ColumnString & column, const char * pos, size_t length)
{
/// String already has terminating null-character.
/// But insertData will add another one unconditionally. Trim existing null-character to avoid duplication.
chassert(0 < length);
chassert(pos[length - 1] == '\0');
column.insertData(pos, length - 1);
}
};
/** For strings. Short strings are stored in the object itself, and long strings are allocated separately. /** For strings. Short strings are stored in the object itself, and long strings are allocated separately.
* NOTE It could also be suitable for arrays of numbers. * NOTE It could also be suitable for arrays of numbers.
@ -477,20 +505,31 @@ public:
return size >= 0; return size >= 0;
} }
const char * getData() const private:
char * getDataMutable()
{ {
return size <= MAX_SMALL_STRING_SIZE ? small_data : large_data; return size <= MAX_SMALL_STRING_SIZE ? small_data : large_data;
} }
const char * getData() const
{
const char * data_ptr = size <= MAX_SMALL_STRING_SIZE ? small_data : large_data;
/// It must always be terminated with null-character
chassert(0 < size);
chassert(data_ptr[size - 1] == '\0');
return data_ptr;
}
StringRef getStringRef() const StringRef getStringRef() const
{ {
return StringRef(getData(), size); return StringRef(getData(), size);
} }
public:
void insertResultInto(IColumn & to) const void insertResultInto(IColumn & to) const
{ {
if (has()) if (has())
assert_cast<ColumnString &>(to).insertData(getData(), size); Compatibility::insertDataWithTerminatingZero(assert_cast<ColumnString &>(to), getData(), size);
else else
assert_cast<ColumnString &>(to).insertDefault(); assert_cast<ColumnString &>(to).insertDefault();
} }
@ -502,44 +541,76 @@ public:
buf.write(getData(), size); buf.write(getData(), size);
} }
void allocateLargeDataIfNeeded(Int64 size_to_reserve, Arena * arena)
{
if (capacity < size_to_reserve)
{
capacity = static_cast<Int32>(roundUpToPowerOfTwoOrZero(size_to_reserve));
/// It might happen if the size was too big and the rounded value does not fit a size_t
if (unlikely(capacity < size_to_reserve))
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "String size is too big ({})", size_to_reserve);
/// Don't free large_data here.
large_data = arena->alloc(capacity);
}
}
void read(ReadBuffer & buf, const ISerialization & /*serialization*/, Arena * arena) void read(ReadBuffer & buf, const ISerialization & /*serialization*/, Arena * arena)
{ {
Int32 rhs_size; Int32 rhs_size;
readBinary(rhs_size, buf); readBinary(rhs_size, buf);
if (rhs_size >= 0) if (rhs_size < 0)
{
if (rhs_size <= MAX_SMALL_STRING_SIZE)
{
/// Don't free large_data here.
size = rhs_size;
if (size > 0)
buf.readStrict(small_data, size);
}
else
{
if (capacity < rhs_size)
{
capacity = static_cast<Int32>(roundUpToPowerOfTwoOrZero(rhs_size));
/// It might happen if the size was too big and the rounded value does not fit a size_t
if (unlikely(capacity < rhs_size))
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "String size is too big ({})", rhs_size);
/// Don't free large_data here.
large_data = arena->alloc(capacity);
}
size = rhs_size;
buf.readStrict(large_data, size);
}
}
else
{ {
/// Don't free large_data here. /// Don't free large_data here.
size = rhs_size; size = rhs_size;
return;
} }
if (rhs_size <= MAX_SMALL_STRING_SIZE)
{
/// Don't free large_data here.
size = rhs_size;
if (size > 0)
buf.readStrict(small_data, size);
}
else
{
/// Reserve one byte more for null-character
Int64 rhs_size_to_reserve = rhs_size;
rhs_size_to_reserve += 1; /// Avoid overflow
allocateLargeDataIfNeeded(rhs_size_to_reserve, arena);
size = rhs_size;
buf.readStrict(large_data, size);
}
/// Check if the string we read is null-terminated (getDataMutable does not have the assertion)
if (0 < size && getDataMutable()[size - 1] == '\0')
return;
/// It's not null-terminated, but it must be (for historical reasons). There are two variants:
/// - The value was serialized by one of the incompatible versions of ClickHouse. We had some range of versions
/// that used to serialize SingleValueDataString without terminating '\0'. Let's just append it.
/// - An attacker sent crafted data. Sanitize it and append '\0'.
/// In all other cases the string must be already null-terminated.
/// NOTE We cannot add '\0' unconditionally, because it will be duplicated.
/// NOTE It's possible that a string that actually ends with '\0' was written by one of the incompatible versions.
/// Unfortunately, we cannot distinguish it from normal string written by normal version.
/// So such strings will be trimmed.
if (size == MAX_SMALL_STRING_SIZE)
{
/// Special case: We have to move value to large_data
allocateLargeDataIfNeeded(size + 1, arena);
memcpy(large_data, small_data, size);
}
/// We have enough space to append
++size;
getDataMutable()[size - 1] = '\0';
} }
/// Assuming to.has() /// Assuming to.has()
@ -557,13 +628,7 @@ public:
} }
else else
{ {
if (capacity < value_size) allocateLargeDataIfNeeded(value_size, arena);
{
/// Don't free large_data here.
capacity = static_cast<Int32>(roundUpToPowerOfTwoOrZero(value_size));
large_data = arena->alloc(capacity);
}
size = value_size; size = value_size;
memcpy(large_data, value.data, size); memcpy(large_data, value.data, size);
} }
@ -571,7 +636,7 @@ public:
void change(const IColumn & column, size_t row_num, Arena * arena) void change(const IColumn & column, size_t row_num, Arena * arena)
{ {
changeImpl(assert_cast<const ColumnString &>(column).getDataAt(row_num), arena); changeImpl(Compatibility::getDataAtWithTerminatingZero(assert_cast<const ColumnString &>(column), row_num), arena);
} }
void change(const Self & to, Arena * arena) void change(const Self & to, Arena * arena)
@ -620,7 +685,7 @@ public:
bool changeIfLess(const IColumn & column, size_t row_num, Arena * arena) bool changeIfLess(const IColumn & column, size_t row_num, Arena * arena)
{ {
if (!has() || assert_cast<const ColumnString &>(column).getDataAt(row_num) < getStringRef()) if (!has() || Compatibility::getDataAtWithTerminatingZero(assert_cast<const ColumnString &>(column), row_num) < getStringRef())
{ {
change(column, row_num, arena); change(column, row_num, arena);
return true; return true;
@ -642,7 +707,7 @@ public:
bool changeIfGreater(const IColumn & column, size_t row_num, Arena * arena) bool changeIfGreater(const IColumn & column, size_t row_num, Arena * arena)
{ {
if (!has() || assert_cast<const ColumnString &>(column).getDataAt(row_num) > getStringRef()) if (!has() || Compatibility::getDataAtWithTerminatingZero(assert_cast<const ColumnString &>(column), row_num) > getStringRef())
{ {
change(column, row_num, arena); change(column, row_num, arena);
return true; return true;
@ -669,7 +734,7 @@ public:
bool isEqualTo(const IColumn & column, size_t row_num) const bool isEqualTo(const IColumn & column, size_t row_num) const
{ {
return has() && assert_cast<const ColumnString &>(column).getDataAt(row_num) == getStringRef(); return has() && Compatibility::getDataAtWithTerminatingZero(assert_cast<const ColumnString &>(column), row_num) == getStringRef();
} }
static bool allocatesMemoryInArena() static bool allocatesMemoryInArena()

View File

@ -15,6 +15,7 @@
#include <Common/formatReadable.h> #include <Common/formatReadable.h>
#include <Common/filesystemHelpers.h> #include <Common/filesystemHelpers.h>
#include <Common/ErrorCodes.h> #include <Common/ErrorCodes.h>
#include <Common/SensitiveDataMasker.h>
#include <Common/LockMemoryExceptionInThread.h> #include <Common/LockMemoryExceptionInThread.h>
#include <filesystem> #include <filesystem>
@ -63,11 +64,18 @@ void handle_error_code([[maybe_unused]] const std::string & msg, int code, bool
ErrorCodes::increment(code, remote, msg, trace); ErrorCodes::increment(code, remote, msg, trace);
} }
Exception::Exception(const std::string & msg, int code, bool remote_) Exception::MessageMasked::MessageMasked(const std::string & msg_)
: Poco::Exception(msg, code) : msg(msg_)
{
if (auto * masker = SensitiveDataMasker::getInstance())
masker->wipeSensitiveData(msg);
}
Exception::Exception(const MessageMasked & msg_masked, int code, bool remote_)
: Poco::Exception(msg_masked.msg, code)
, remote(remote_) , remote(remote_)
{ {
handle_error_code(msg, code, remote, getStackFramePointers()); handle_error_code(msg_masked.msg, code, remote, getStackFramePointers());
} }
Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc) Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc)

View File

@ -27,7 +27,19 @@ public:
using FramePointers = std::vector<void *>; using FramePointers = std::vector<void *>;
Exception() = default; Exception() = default;
Exception(const std::string & msg, int code, bool remote_ = false);
// used to remove the sensitive information from exceptions if query_masking_rules is configured
struct MessageMasked
{
std::string msg;
MessageMasked(const std::string & msg_);
};
Exception(const MessageMasked & msg_masked, int code, bool remote_);
// delegating constructor to mask sensitive information from the message
Exception(const std::string & msg, int code, bool remote_ = false): Exception(MessageMasked(msg), code, remote_)
{}
Exception(int code, const std::string & message) Exception(int code, const std::string & message)
: Exception(message, code) : Exception(message, code)
@ -54,12 +66,17 @@ public:
template <typename... Args> template <typename... Args>
void addMessage(fmt::format_string<Args...> format, Args &&... args) void addMessage(fmt::format_string<Args...> format, Args &&... args)
{ {
extendedMessage(fmt::format(format, std::forward<Args>(args)...)); addMessage(fmt::format(format, std::forward<Args>(args)...));
} }
void addMessage(const std::string& message) void addMessage(const std::string& message)
{ {
extendedMessage(message); addMessage(MessageMasked(message));
}
void addMessage(const MessageMasked & msg_masked)
{
extendedMessage(msg_masked.msg);
} }
/// Used to distinguish local exceptions from the one that was received from remote node. /// Used to distinguish local exceptions from the one that was received from remote node.

View File

@ -220,7 +220,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT
Int64 limit_to_check = current_hard_limit; Int64 limit_to_check = current_hard_limit;
#if USE_JEMALLOC #if USE_JEMALLOC
if (level == VariableContext::Global) if (level == VariableContext::Global && allow_use_jemalloc_memory.load(std::memory_order_relaxed))
{ {
/// Jemalloc arenas may keep some extra memory. /// Jemalloc arenas may keep some extra memory.
/// This memory was substucted from RSS to decrease memory drift. /// This memory was substucted from RSS to decrease memory drift.

View File

@ -55,6 +55,7 @@ private:
std::atomic<Int64> soft_limit {0}; std::atomic<Int64> soft_limit {0};
std::atomic<Int64> hard_limit {0}; std::atomic<Int64> hard_limit {0};
std::atomic<Int64> profiler_limit {0}; std::atomic<Int64> profiler_limit {0};
std::atomic_bool allow_use_jemalloc_memory {true};
static std::atomic<Int64> free_memory_in_allocator_arenas; static std::atomic<Int64> free_memory_in_allocator_arenas;
@ -125,6 +126,10 @@ public:
{ {
return soft_limit.load(std::memory_order_relaxed); return soft_limit.load(std::memory_order_relaxed);
} }
void setAllowUseJemallocMemory(bool value)
{
allow_use_jemalloc_memory.store(value, std::memory_order_relaxed);
}
/** Set limit if it was not set. /** Set limit if it was not set.
* Otherwise, set limit to new value, if new value is greater than previous limit. * Otherwise, set limit to new value, if new value is greater than previous limit.

View File

@ -135,6 +135,9 @@ void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_
if (source_size < 2) if (source_size < 2)
throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS);
if (uncompressed_size == 0)
return;
UInt8 bytes_size = source[0]; UInt8 bytes_size = source[0];
if (bytes_size == 0) if (bytes_size == 0)

View File

@ -36,7 +36,7 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco
} }
const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif"; const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld";
KeeperConfigurationAndSettings::KeeperConfigurationAndSettings() KeeperConfigurationAndSettings::KeeperConfigurationAndSettings()
: server_id(NOT_EXIST) : server_id(NOT_EXIST)

View File

@ -142,6 +142,9 @@ void FourLetterCommandFactory::registerCommands(KeeperDispatcher & keeper_dispat
FourLetterCommandPtr log_info_command = std::make_shared<LogInfoCommand>(keeper_dispatcher); FourLetterCommandPtr log_info_command = std::make_shared<LogInfoCommand>(keeper_dispatcher);
factory.registerCommand(log_info_command); factory.registerCommand(log_info_command);
FourLetterCommandPtr request_leader_command = std::make_shared<RequestLeaderCommand>(keeper_dispatcher);
factory.registerCommand(request_leader_command);
factory.initializeAllowList(keeper_dispatcher); factory.initializeAllowList(keeper_dispatcher);
factory.setInitialize(true); factory.setInitialize(true);
} }
@ -507,4 +510,9 @@ String LogInfoCommand::run()
return ret.str(); return ret.str();
} }
String RequestLeaderCommand::run()
{
return keeper_dispatcher.requestLeader() ? "Sent leadership request to leader." : "Failed to send leadership request to leader.";
}
} }

View File

@ -364,4 +364,17 @@ struct LogInfoCommand : public IFourLetterCommand
~LogInfoCommand() override = default; ~LogInfoCommand() override = default;
}; };
/// Request to be leader.
struct RequestLeaderCommand : public IFourLetterCommand
{
explicit RequestLeaderCommand(KeeperDispatcher & keeper_dispatcher_)
: IFourLetterCommand(keeper_dispatcher_)
{
}
String name() override { return "rqld"; }
String run() override;
~RequestLeaderCommand() override = default;
};
} }

View File

@ -215,6 +215,12 @@ public:
{ {
return server->getKeeperLogInfo(); return server->getKeeperLogInfo();
} }
/// Request to be leader.
bool requestLeader()
{
return server->requestLeader();
}
}; };
} }

View File

@ -932,4 +932,9 @@ KeeperLogInfo KeeperServer::getKeeperLogInfo()
return log_info; return log_info;
} }
bool KeeperServer::requestLeader()
{
return isLeader() || raft_instance->request_leadership();
}
} }

View File

@ -135,6 +135,8 @@ public:
uint64_t createSnapshot(); uint64_t createSnapshot();
KeeperLogInfo getKeeperLogInfo(); KeeperLogInfo getKeeperLogInfo();
bool requestLeader();
}; };
} }

View File

@ -4,7 +4,10 @@
namespace DB namespace DB
{ {
DiskDecorator::DiskDecorator(const DiskPtr & delegate_) : delegate(delegate_)
DiskDecorator::DiskDecorator(const DiskPtr & delegate_)
: IDisk(/* name_= */ "<decorator>")
, delegate(delegate_)
{ {
} }
@ -226,9 +229,9 @@ void DiskDecorator::shutdown()
delegate->shutdown(); delegate->shutdown();
} }
void DiskDecorator::startup(ContextPtr context) void DiskDecorator::startupImpl(ContextPtr context)
{ {
delegate->startup(context); delegate->startupImpl(context);
} }
void DiskDecorator::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap & map) void DiskDecorator::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap & map)

View File

@ -74,12 +74,14 @@ public:
bool checkUniqueId(const String & id) const override { return delegate->checkUniqueId(id); } bool checkUniqueId(const String & id) const override { return delegate->checkUniqueId(id); }
DataSourceDescription getDataSourceDescription() const override { return delegate->getDataSourceDescription(); } DataSourceDescription getDataSourceDescription() const override { return delegate->getDataSourceDescription(); }
bool isRemote() const override { return delegate->isRemote(); } bool isRemote() const override { return delegate->isRemote(); }
bool isReadOnly() const override { return delegate->isReadOnly(); }
bool isWriteOnce() const override { return delegate->isWriteOnce(); }
bool supportZeroCopyReplication() const override { return delegate->supportZeroCopyReplication(); } bool supportZeroCopyReplication() const override { return delegate->supportZeroCopyReplication(); }
bool supportParallelWrite() const override { return delegate->supportParallelWrite(); } bool supportParallelWrite() const override { return delegate->supportParallelWrite(); }
void onFreeze(const String & path) override; void onFreeze(const String & path) override;
SyncGuardPtr getDirectorySyncGuard(const String & path) const override; SyncGuardPtr getDirectorySyncGuard(const String & path) const override;
void shutdown() override; void shutdown() override;
void startup(ContextPtr context) override; void startupImpl(ContextPtr context) override;
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap & map) override; void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap & map) override;
bool supportsCache() const override { return delegate->supportsCache(); } bool supportsCache() const override { return delegate->supportsCache(); }

View File

@ -210,7 +210,7 @@ DiskEncrypted::DiskEncrypted(
DiskEncrypted::DiskEncrypted(const String & name_, std::unique_ptr<const DiskEncryptedSettings> settings_) DiskEncrypted::DiskEncrypted(const String & name_, std::unique_ptr<const DiskEncryptedSettings> settings_)
: DiskDecorator(settings_->wrapped_disk) : DiskDecorator(settings_->wrapped_disk)
, name(name_) , encrypted_name(name_)
, disk_path(settings_->disk_path) , disk_path(settings_->disk_path)
, disk_absolute_path(settings_->wrapped_disk->getPath() + settings_->disk_path) , disk_absolute_path(settings_->wrapped_disk->getPath() + settings_->disk_path)
, current_settings(std::move(settings_)) , current_settings(std::move(settings_))
@ -369,15 +369,19 @@ void DiskEncrypted::applyNewSettings(
current_settings.set(std::move(new_settings)); current_settings.set(std::move(new_settings));
} }
void registerDiskEncrypted(DiskFactory & factory) void registerDiskEncrypted(DiskFactory & factory, bool global_skip_access_check)
{ {
auto creator = [](const String & name, auto creator = [global_skip_access_check](
const Poco::Util::AbstractConfiguration & config, const String & name,
const String & config_prefix, const Poco::Util::AbstractConfiguration & config,
ContextPtr /*context*/, const String & config_prefix,
const DisksMap & map) -> DiskPtr ContextPtr context,
const DisksMap & map) -> DiskPtr
{ {
return std::make_shared<DiskEncrypted>(name, config, config_prefix, map); bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false);
DiskPtr disk = std::make_shared<DiskEncrypted>(name, config, config_prefix, map);
disk->startup(context, skip_access_check);
return disk;
}; };
factory.registerDiskType("encrypted", creator); factory.registerDiskType("encrypted", creator);
} }

View File

@ -33,7 +33,7 @@ public:
DiskEncrypted(const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_); DiskEncrypted(const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_);
DiskEncrypted(const String & name_, std::unique_ptr<const DiskEncryptedSettings> settings_); DiskEncrypted(const String & name_, std::unique_ptr<const DiskEncryptedSettings> settings_);
const String & getName() const override { return name; } const String & getName() const override { return encrypted_name; }
const String & getPath() const override { return disk_absolute_path; } const String & getPath() const override { return disk_absolute_path; }
ReservationPtr reserve(UInt64 bytes) override; ReservationPtr reserve(UInt64 bytes) override;
@ -261,7 +261,7 @@ private:
return disk_path + path; return disk_path + path;
} }
const String name; const String encrypted_name;
const String disk_path; const String disk_path;
const String disk_absolute_path; const String disk_absolute_path;
MultiVersion<DiskEncryptedSettings> current_settings; MultiVersion<DiskEncryptedSettings> current_settings;

View File

@ -500,7 +500,7 @@ void DiskLocal::applyNewSettings(const Poco::Util::AbstractConfiguration & confi
} }
DiskLocal::DiskLocal(const String & name_, const String & path_, UInt64 keep_free_space_bytes_) DiskLocal::DiskLocal(const String & name_, const String & path_, UInt64 keep_free_space_bytes_)
: name(name_) : IDisk(name_)
, disk_path(path_) , disk_path(path_)
, keep_free_space_bytes(keep_free_space_bytes_) , keep_free_space_bytes(keep_free_space_bytes_)
, logger(&Poco::Logger::get("DiskLocal")) , logger(&Poco::Logger::get("DiskLocal"))
@ -528,26 +528,6 @@ DataSourceDescription DiskLocal::getDataSourceDescription() const
return data_source_description; return data_source_description;
} }
void DiskLocal::startup(ContextPtr)
{
try
{
broken = false;
disk_checker_magic_number = -1;
disk_checker_can_check_read = true;
readonly = !setup();
}
catch (...)
{
tryLogCurrentException(logger, fmt::format("Disk {} is marked as broken during startup", name));
broken = true;
/// Disk checker is disabled when failing to start up.
disk_checker_can_check_read = false;
}
if (disk_checker && disk_checker_can_check_read)
disk_checker->startup();
}
void DiskLocal::shutdown() void DiskLocal::shutdown()
{ {
if (disk_checker) if (disk_checker)
@ -641,18 +621,30 @@ DiskObjectStoragePtr DiskLocal::createDiskObjectStorage()
); );
} }
bool DiskLocal::setup() void DiskLocal::checkAccessImpl(const String & path)
{ {
try try
{ {
fs::create_directories(disk_path); fs::create_directories(disk_path);
if (!FS::canWrite(disk_path))
{
LOG_ERROR(logger, "Cannot write to the root directory of disk {} ({}).", name, disk_path);
readonly = true;
return;
}
} }
catch (...) catch (...)
{ {
LOG_ERROR(logger, "Cannot create the directory of disk {} ({}).", name, disk_path); LOG_ERROR(logger, "Cannot create the root directory of disk {} ({}).", name, disk_path);
throw; readonly = true;
return;
} }
IDisk::checkAccessImpl(path);
}
void DiskLocal::setup()
{
try try
{ {
if (!FS::canRead(disk_path)) if (!FS::canRead(disk_path))
@ -666,7 +658,7 @@ bool DiskLocal::setup()
/// If disk checker is disabled, just assume RW by default. /// If disk checker is disabled, just assume RW by default.
if (!disk_checker) if (!disk_checker)
return true; return;
try try
{ {
@ -690,6 +682,7 @@ bool DiskLocal::setup()
/// Try to create a new checker file. The disk status can be either broken or readonly. /// Try to create a new checker file. The disk status can be either broken or readonly.
if (disk_checker_magic_number == -1) if (disk_checker_magic_number == -1)
{
try try
{ {
pcg32_fast rng(randomSeed()); pcg32_fast rng(randomSeed());
@ -709,12 +702,33 @@ bool DiskLocal::setup()
disk_checker_path, disk_checker_path,
name); name);
disk_checker_can_check_read = false; disk_checker_can_check_read = false;
return true; return;
} }
}
if (disk_checker_magic_number == -1) if (disk_checker_magic_number == -1)
throw Exception("disk_checker_magic_number is not initialized. It's a bug", ErrorCodes::LOGICAL_ERROR); throw Exception("disk_checker_magic_number is not initialized. It's a bug", ErrorCodes::LOGICAL_ERROR);
return true; }
void DiskLocal::startupImpl(ContextPtr)
{
broken = false;
disk_checker_magic_number = -1;
disk_checker_can_check_read = true;
try
{
setup();
}
catch (...)
{
tryLogCurrentException(logger, fmt::format("Disk {} is marked as broken during startup", name));
broken = true;
/// Disk checker is disabled when failing to start up.
disk_checker_can_check_read = false;
}
if (disk_checker && disk_checker_can_check_read)
disk_checker->startup();
} }
struct stat DiskLocal::stat(const String & path) const struct stat DiskLocal::stat(const String & path) const
@ -741,13 +755,14 @@ MetadataStoragePtr DiskLocal::getMetadataStorage()
std::static_pointer_cast<IDisk>(shared_from_this()), object_storage, getPath()); std::static_pointer_cast<IDisk>(shared_from_this()), object_storage, getPath());
} }
void registerDiskLocal(DiskFactory & factory) void registerDiskLocal(DiskFactory & factory, bool global_skip_access_check)
{ {
auto creator = [](const String & name, auto creator = [global_skip_access_check](
const Poco::Util::AbstractConfiguration & config, const String & name,
const String & config_prefix, const Poco::Util::AbstractConfiguration & config,
ContextPtr context, const String & config_prefix,
const DisksMap & map) -> DiskPtr ContextPtr context,
const DisksMap & map) -> DiskPtr
{ {
String path; String path;
UInt64 keep_free_space_bytes; UInt64 keep_free_space_bytes;
@ -757,9 +772,10 @@ void registerDiskLocal(DiskFactory & factory)
if (path == disk_ptr->getPath()) if (path == disk_ptr->getPath())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk {} and disk {} cannot have the same path ({})", name, disk_name, path); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk {} and disk {} cannot have the same path ({})", name, disk_name, path);
bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false);
std::shared_ptr<IDisk> disk std::shared_ptr<IDisk> disk
= std::make_shared<DiskLocal>(name, path, keep_free_space_bytes, context, config.getUInt("local_disk_check_period_ms", 0)); = std::make_shared<DiskLocal>(name, path, keep_free_space_bytes, context, config.getUInt("local_disk_check_period_ms", 0));
disk->startup(context); disk->startup(context, skip_access_check);
return std::make_shared<DiskRestartProxy>(disk); return std::make_shared<DiskRestartProxy>(disk);
}; };
factory.registerDiskType("local", creator); factory.registerDiskType("local", creator);

View File

@ -28,8 +28,6 @@ public:
ContextPtr context, ContextPtr context,
UInt64 local_disk_check_period_ms); UInt64 local_disk_check_period_ms);
const String & getName() const override { return name; }
const String & getPath() const override { return disk_path; } const String & getPath() const override { return disk_path; }
ReservationPtr reserve(UInt64 bytes) override; ReservationPtr reserve(UInt64 bytes) override;
@ -112,8 +110,9 @@ public:
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap &) override; void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap &) override;
bool isBroken() const override { return broken; } bool isBroken() const override { return broken; }
bool isReadOnly() const override { return readonly; }
void startup(ContextPtr) override; void startupImpl(ContextPtr context) override;
void shutdown() override; void shutdown() override;
@ -133,17 +132,19 @@ public:
MetadataStoragePtr getMetadataStorage() override; MetadataStoragePtr getMetadataStorage() override;
protected:
void checkAccessImpl(const String & path) override;
private: private:
std::optional<UInt64> tryReserve(UInt64 bytes); std::optional<UInt64> tryReserve(UInt64 bytes);
/// Setup disk for healthy check. Returns true if it's read-write, false if read-only. /// Setup disk for healthy check.
/// Throw exception if it's not possible to setup necessary files and directories. /// Throw exception if it's not possible to setup necessary files and directories.
bool setup(); void setup();
/// Read magic number from disk checker file. Return std::nullopt if exception happens. /// Read magic number from disk checker file. Return std::nullopt if exception happens.
std::optional<UInt32> readDiskCheckerMagicNumber() const noexcept; std::optional<UInt32> readDiskCheckerMagicNumber() const noexcept;
const String name;
const String disk_path; const String disk_path;
const String disk_checker_path = ".disk_checker_file"; const String disk_checker_path = ".disk_checker_file";
std::atomic<UInt64> keep_free_space_bytes; std::atomic<UInt64> keep_free_space_bytes;

View File

@ -141,6 +141,11 @@ private:
}; };
DiskMemory::DiskMemory(const String & name_)
: IDisk(name_)
, disk_path("memory(" + name_ + ')')
{}
ReservationPtr DiskMemory::reserve(UInt64 /*bytes*/) ReservationPtr DiskMemory::reserve(UInt64 /*bytes*/)
{ {
throw Exception("Method reserve is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); throw Exception("Method reserve is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED);
@ -456,13 +461,20 @@ MetadataStoragePtr DiskMemory::getMetadataStorage()
using DiskMemoryPtr = std::shared_ptr<DiskMemory>; using DiskMemoryPtr = std::shared_ptr<DiskMemory>;
void registerDiskMemory(DiskFactory & factory) void registerDiskMemory(DiskFactory & factory, bool global_skip_access_check)
{ {
auto creator = [](const String & name, auto creator = [global_skip_access_check](
const Poco::Util::AbstractConfiguration & /*config*/, const String & name,
const String & /*config_prefix*/, const Poco::Util::AbstractConfiguration & config,
ContextPtr /*context*/, const String & config_prefix,
const DisksMap & /*map*/) -> DiskPtr { return std::make_shared<DiskMemory>(name); }; ContextPtr context,
const DisksMap & /*map*/) -> DiskPtr
{
bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false);
DiskPtr disk = std::make_shared<DiskMemory>(name);
disk->startup(context, skip_access_check);
return disk;
};
factory.registerDiskType("memory", creator); factory.registerDiskType("memory", creator);
} }

View File

@ -8,7 +8,7 @@
namespace DB namespace DB
{ {
class DiskMemory;
class ReadBufferFromFileBase; class ReadBufferFromFileBase;
class WriteBufferFromFileBase; class WriteBufferFromFileBase;
@ -22,9 +22,7 @@ class WriteBufferFromFileBase;
class DiskMemory : public IDisk class DiskMemory : public IDisk
{ {
public: public:
explicit DiskMemory(const String & name_) : name(name_), disk_path("memory://" + name_ + '/') {} explicit DiskMemory(const String & name_);
const String & getName() const override { return name; }
const String & getPath() const override { return disk_path; } const String & getPath() const override { return disk_path; }
@ -121,7 +119,6 @@ private:
}; };
using Files = std::unordered_map<String, FileData>; /// file path -> file data using Files = std::unordered_map<String, FileData>; /// file path -> file data
const String name;
const String disk_path; const String disk_path;
Files files; Files files;
mutable std::mutex mutex; mutable std::mutex mutex;

View File

@ -79,7 +79,8 @@ private:
}; };
DiskRestartProxy::DiskRestartProxy(DiskPtr & delegate_) DiskRestartProxy::DiskRestartProxy(DiskPtr & delegate_)
: DiskDecorator(delegate_) { } : DiskDecorator(delegate_)
{}
ReservationPtr DiskRestartProxy::reserve(UInt64 bytes) ReservationPtr DiskRestartProxy::reserve(UInt64 bytes)
{ {
@ -368,7 +369,8 @@ void DiskRestartProxy::restart(ContextPtr context)
LOG_INFO(log, "Restart lock acquired. Restarting disk {}", DiskDecorator::getName()); LOG_INFO(log, "Restart lock acquired. Restarting disk {}", DiskDecorator::getName());
DiskDecorator::startup(context); /// NOTE: access checking will cause deadlock here, so skip it.
DiskDecorator::startup(context, /* skip_access_check= */ true);
LOG_INFO(log, "Disk restarted {}", DiskDecorator::getName()); LOG_INFO(log, "Disk restarted {}", DiskDecorator::getName());
} }

View File

@ -6,6 +6,7 @@
#include <Poco/Logger.h> #include <Poco/Logger.h>
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <Core/ServerUUID.h>
#include <Disks/ObjectStorages/MetadataStorageFromDisk.h> #include <Disks/ObjectStorages/MetadataStorageFromDisk.h>
#include <Disks/ObjectStorages/FakeMetadataStorageFromDisk.h> #include <Disks/ObjectStorages/FakeMetadataStorageFromDisk.h>
#include <Disks/ObjectStorages/LocalObjectStorage.h> #include <Disks/ObjectStorages/LocalObjectStorage.h>
@ -17,6 +18,8 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int NOT_IMPLEMENTED; extern const int NOT_IMPLEMENTED;
extern const int CANNOT_READ_ALL_DATA;
extern const int LOGICAL_ERROR;
} }
bool IDisk::isDirectoryEmpty(const String & path) const bool IDisk::isDirectoryEmpty(const String & path) const
@ -126,4 +129,87 @@ SyncGuardPtr IDisk::getDirectorySyncGuard(const String & /* path */) const
return nullptr; return nullptr;
} }
void IDisk::startup(ContextPtr context, bool skip_access_check)
{
if (!skip_access_check)
{
if (isReadOnly())
{
LOG_DEBUG(&Poco::Logger::get("IDisk"),
"Skip access check for disk {} (read-only disk).",
getName());
}
else
checkAccess();
}
startupImpl(context);
}
void IDisk::checkAccess()
{
DB::UUID server_uuid = DB::ServerUUID::get();
if (server_uuid == DB::UUIDHelpers::Nil)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Server UUID is not initialized");
const String path = fmt::format("clickhouse_access_check_{}", DB::toString(server_uuid));
checkAccessImpl(path);
}
/// NOTE: should we mark the disk readonly if the write/unlink fails instead of throws?
void IDisk::checkAccessImpl(const String & path)
try
{
const std::string_view payload("test", 4);
/// write
{
auto file = writeFile(path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite);
try
{
file->write(payload.data(), payload.size());
}
catch (...)
{
/// Log current exception, because finalize() can throw a different exception.
tryLogCurrentException(__PRETTY_FUNCTION__);
file->finalize();
throw;
}
}
/// read
{
auto file = readFile(path);
String buf(payload.size(), '0');
file->readStrict(buf.data(), buf.size());
if (buf != payload)
{
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA,
"Content of {}::{} does not matches after read ({} vs {})", name, path, buf, payload);
}
}
/// read with offset
{
auto file = readFile(path);
auto offset = 2;
String buf(payload.size() - offset, '0');
file->seek(offset, 0);
file->readStrict(buf.data(), buf.size());
if (buf != payload.substr(offset))
{
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA,
"Content of {}::{} does not matches after read with offset ({} vs {})", name, path, buf, payload.substr(offset));
}
}
/// remove
removeFile(path);
}
catch (Exception & e)
{
e.addMessage(fmt::format("While checking access for disk {}", name));
throw;
}
} }

View File

@ -107,8 +107,9 @@ class IDisk : public Space
{ {
public: public:
/// Default constructor. /// Default constructor.
explicit IDisk(std::shared_ptr<Executor> executor_ = std::make_shared<SyncExecutor>()) explicit IDisk(const String & name_, std::shared_ptr<Executor> executor_ = std::make_shared<SyncExecutor>())
: executor(executor_) : name(name_)
, executor(executor_)
{ {
} }
@ -121,6 +122,9 @@ public:
/// It's not required to be a local filesystem path. /// It's not required to be a local filesystem path.
virtual const String & getPath() const = 0; virtual const String & getPath() const = 0;
/// Return disk name.
const String & getName() const override { return name; }
/// Total available space on the disk. /// Total available space on the disk.
virtual UInt64 getTotalSpace() const = 0; virtual UInt64 getTotalSpace() const = 0;
@ -308,14 +312,19 @@ public:
virtual bool isReadOnly() const { return false; } virtual bool isReadOnly() const { return false; }
virtual bool isWriteOnce() const { return false; }
/// Check if disk is broken. Broken disks will have 0 space and cannot be used. /// Check if disk is broken. Broken disks will have 0 space and cannot be used.
virtual bool isBroken() const { return false; } virtual bool isBroken() const { return false; }
/// Invoked when Global Context is shutdown. /// Invoked when Global Context is shutdown.
virtual void shutdown() {} virtual void shutdown() {}
/// Performs action on disk startup. /// Performs access check and custom action on disk startup.
virtual void startup(ContextPtr) {} void startup(ContextPtr context, bool skip_access_check);
/// Performs custom action on disk startup.
virtual void startupImpl(ContextPtr) {}
/// Return some uniq string for file, overrode for IDiskRemote /// Return some uniq string for file, overrode for IDiskRemote
/// Required for distinguish different copies of the same part on remote disk /// Required for distinguish different copies of the same part on remote disk
@ -403,6 +412,8 @@ public:
protected: protected:
friend class DiskDecorator; friend class DiskDecorator;
const String name;
/// Returns executor to perform asynchronous operations. /// Returns executor to perform asynchronous operations.
virtual Executor & getExecutor() { return *executor; } virtual Executor & getExecutor() { return *executor; }
@ -411,9 +422,14 @@ protected:
/// A derived class may override copy() to provide a faster implementation. /// A derived class may override copy() to provide a faster implementation.
void copyThroughBuffers(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path, bool copy_root_dir = true); void copyThroughBuffers(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path, bool copy_root_dir = true);
virtual void checkAccessImpl(const String & path);
private: private:
std::shared_ptr<Executor> executor; std::shared_ptr<Executor> executor;
bool is_custom_disk = false; bool is_custom_disk = false;
/// Check access to the disk.
void checkAccess();
}; };
using Disks = std::vector<DiskPtr>; using Disks = std::vector<DiskPtr>;

View File

@ -17,55 +17,9 @@
namespace DB namespace DB
{ {
namespace ErrorCodes void registerDiskAzureBlobStorage(DiskFactory & factory, bool global_skip_access_check)
{ {
extern const int PATH_ACCESS_DENIED; auto creator = [global_skip_access_check](
}
namespace
{
constexpr char test_file[] = "test.txt";
constexpr char test_str[] = "test";
constexpr size_t test_str_size = 4;
void checkWriteAccess(IDisk & disk)
{
auto file = disk.writeFile(test_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite);
file->write(test_str, test_str_size);
}
void checkReadAccess(IDisk & disk)
{
auto file = disk.readFile(test_file);
String buf(test_str_size, '0');
file->readStrict(buf.data(), test_str_size);
if (buf != test_str)
throw Exception("No read access to disk", ErrorCodes::PATH_ACCESS_DENIED);
}
void checkReadWithOffset(IDisk & disk)
{
auto file = disk.readFile(test_file);
auto offset = 2;
auto test_size = test_str_size - offset;
String buf(test_size, '0');
file->seek(offset, 0);
file->readStrict(buf.data(), test_size);
if (buf != test_str + offset)
throw Exception("Failed to read file with offset", ErrorCodes::PATH_ACCESS_DENIED);
}
void checkRemoveAccess(IDisk & disk)
{
disk.removeFile(test_file);
}
}
void registerDiskAzureBlobStorage(DiskFactory & factory)
{
auto creator = [](
const String & name, const String & name,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const String & config_prefix, const String & config_prefix,
@ -94,15 +48,8 @@ void registerDiskAzureBlobStorage(DiskFactory & factory)
copy_thread_pool_size copy_thread_pool_size
); );
if (!config.getBool(config_prefix + ".skip_access_check", false)) bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false);
{ azure_blob_storage_disk->startup(context, skip_access_check);
checkWriteAccess(*azure_blob_storage_disk);
checkReadAccess(*azure_blob_storage_disk);
checkReadWithOffset(*azure_blob_storage_disk);
checkRemoveAccess(*azure_blob_storage_disk);
}
azure_blob_storage_disk->startup(context);
return std::make_shared<DiskRestartProxy>(azure_blob_storage_disk); return std::make_shared<DiskRestartProxy>(azure_blob_storage_disk);
}; };
@ -117,7 +64,7 @@ void registerDiskAzureBlobStorage(DiskFactory & factory)
namespace DB namespace DB
{ {
void registerDiskAzureBlobStorage(DiskFactory &) {} void registerDiskAzureBlobStorage(DiskFactory &, bool /* global_skip_access_check */) {}
} }

View File

@ -101,6 +101,8 @@ public:
bool isReadOnly() const override { return object_storage->isReadOnly(); } bool isReadOnly() const override { return object_storage->isReadOnly(); }
bool isWriteOnce() const override { return object_storage->isWriteOnce(); }
const std::string & getCacheConfigName() const { return cache_config_name; } const std::string & getCacheConfigName() const { return cache_config_name; }
ObjectStoragePtr getWrappedObjectStorage() { return object_storage; } ObjectStoragePtr getWrappedObjectStorage() { return object_storage; }

View File

@ -16,7 +16,7 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
} }
void registerDiskCache(DiskFactory & factory) void registerDiskCache(DiskFactory & factory, bool /* global_skip_access_check */)
{ {
auto creator = [](const String & name, auto creator = [](const String & name,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,

View File

@ -109,8 +109,7 @@ DiskObjectStorage::DiskObjectStorage(
ObjectStoragePtr object_storage_, ObjectStoragePtr object_storage_,
bool send_metadata_, bool send_metadata_,
uint64_t thread_pool_size_) uint64_t thread_pool_size_)
: IDisk(getAsyncExecutor(log_name, thread_pool_size_)) : IDisk(name_, getAsyncExecutor(log_name, thread_pool_size_))
, name(name_)
, object_storage_root_path(object_storage_root_path_) , object_storage_root_path(object_storage_root_path_)
, log (&Poco::Logger::get("DiskObjectStorage(" + log_name + ")")) , log (&Poco::Logger::get("DiskObjectStorage(" + log_name + ")"))
, metadata_storage(std::move(metadata_storage_)) , metadata_storage(std::move(metadata_storage_))
@ -420,9 +419,8 @@ void DiskObjectStorage::shutdown()
LOG_INFO(log, "Disk {} shut down", name); LOG_INFO(log, "Disk {} shut down", name);
} }
void DiskObjectStorage::startup(ContextPtr context) void DiskObjectStorage::startupImpl(ContextPtr context)
{ {
LOG_INFO(log, "Starting up disk {}", name); LOG_INFO(log, "Starting up disk {}", name);
object_storage->startup(); object_storage->startup();
@ -499,6 +497,11 @@ bool DiskObjectStorage::isReadOnly() const
return object_storage->isReadOnly(); return object_storage->isReadOnly();
} }
bool DiskObjectStorage::isWriteOnce() const
{
return object_storage->isWriteOnce();
}
DiskObjectStoragePtr DiskObjectStorage::createDiskObjectStorage() DiskObjectStoragePtr DiskObjectStorage::createDiskObjectStorage()
{ {
return std::make_shared<DiskObjectStorage>( return std::make_shared<DiskObjectStorage>(

View File

@ -45,8 +45,6 @@ public:
bool supportParallelWrite() const override { return object_storage->supportParallelWrite(); } bool supportParallelWrite() const override { return object_storage->supportParallelWrite(); }
const String & getName() const override { return name; }
const String & getPath() const override { return metadata_storage->getPath(); } const String & getPath() const override { return metadata_storage->getPath(); }
StoredObjects getStorageObjects(const String & local_path) const override; StoredObjects getStorageObjects(const String & local_path) const override;
@ -138,7 +136,7 @@ public:
void shutdown() override; void shutdown() override;
void startup(ContextPtr context) override; void startupImpl(ContextPtr context) override;
ReservationPtr reserve(UInt64 bytes) override; ReservationPtr reserve(UInt64 bytes) override;
@ -177,6 +175,12 @@ public:
/// with static files, so only read-only operations are allowed for this storage. /// with static files, so only read-only operations are allowed for this storage.
bool isReadOnly() const override; bool isReadOnly() const override;
/// Is object write-once?
/// For example: S3PlainObjectStorage is write once, this means that it
/// does support BACKUP to this disk, but does not support INSERT into
/// MergeTree table on this disk.
bool isWriteOnce() const override;
/// Add a cache layer. /// Add a cache layer.
/// Example: DiskObjectStorage(S3ObjectStorage) -> DiskObjectStorage(CachedObjectStorage(S3ObjectStorage)) /// Example: DiskObjectStorage(S3ObjectStorage) -> DiskObjectStorage(CachedObjectStorage(S3ObjectStorage))
/// There can be any number of cache layers: /// There can be any number of cache layers:
@ -206,7 +210,6 @@ private:
/// execution. /// execution.
DiskTransactionPtr createObjectStorageTransaction(); DiskTransactionPtr createObjectStorageTransaction();
const String name;
const String object_storage_root_path; const String object_storage_root_path;
Poco::Logger * log; Poco::Logger * log;

View File

@ -15,13 +15,14 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
} }
void registerDiskHDFS(DiskFactory & factory) void registerDiskHDFS(DiskFactory & factory, bool global_skip_access_check)
{ {
auto creator = [](const String & name, auto creator = [global_skip_access_check](
const Poco::Util::AbstractConfiguration & config, const String & name,
const String & config_prefix, const Poco::Util::AbstractConfiguration & config,
ContextPtr context_, const String & config_prefix,
const DisksMap & /*map*/) -> DiskPtr ContextPtr context,
const DisksMap & /*map*/) -> DiskPtr
{ {
String uri{config.getString(config_prefix + ".endpoint")}; String uri{config.getString(config_prefix + ".endpoint")};
checkHDFSURL(uri); checkHDFSURL(uri);
@ -32,19 +33,20 @@ void registerDiskHDFS(DiskFactory & factory)
std::unique_ptr<HDFSObjectStorageSettings> settings = std::make_unique<HDFSObjectStorageSettings>( std::unique_ptr<HDFSObjectStorageSettings> settings = std::make_unique<HDFSObjectStorageSettings>(
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024),
config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000), config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000),
context_->getSettingsRef().hdfs_replication context->getSettingsRef().hdfs_replication
); );
/// FIXME Cache currently unsupported :( /// FIXME Cache currently unsupported :(
ObjectStoragePtr hdfs_storage = std::make_unique<HDFSObjectStorage>(uri, std::move(settings), config); ObjectStoragePtr hdfs_storage = std::make_unique<HDFSObjectStorage>(uri, std::move(settings), config);
auto [_, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context_->getPath()); auto [_, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context->getPath());
auto metadata_storage = std::make_shared<MetadataStorageFromDisk>(metadata_disk, uri); auto metadata_storage = std::make_shared<MetadataStorageFromDisk>(metadata_disk, uri);
uint64_t copy_thread_pool_size = config.getUInt(config_prefix + ".thread_pool_size", 16); uint64_t copy_thread_pool_size = config.getUInt(config_prefix + ".thread_pool_size", 16);
bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false);
DiskPtr disk_result = std::make_shared<DiskObjectStorage>( DiskPtr disk = std::make_shared<DiskObjectStorage>(
name, name,
uri, uri,
"DiskHDFS", "DiskHDFS",
@ -52,8 +54,9 @@ void registerDiskHDFS(DiskFactory & factory)
std::move(hdfs_storage), std::move(hdfs_storage),
/* send_metadata = */ false, /* send_metadata = */ false,
copy_thread_pool_size); copy_thread_pool_size);
disk->startup(context, skip_access_check);
return std::make_shared<DiskRestartProxy>(disk_result); return std::make_shared<DiskRestartProxy>(disk);
}; };
factory.registerDiskType("hdfs", creator); factory.registerDiskType("hdfs", creator);

View File

@ -199,6 +199,7 @@ public:
virtual bool supportsCache() const { return false; } virtual bool supportsCache() const { return false; }
virtual bool isReadOnly() const { return false; } virtual bool isReadOnly() const { return false; }
virtual bool isWriteOnce() const { return false; }
virtual bool supportParallelWrite() const { return false; } virtual bool supportParallelWrite() const { return false; }

View File

@ -216,6 +216,11 @@ public:
{ {
data_source_description.type = DataSourceType::S3_Plain; data_source_description.type = DataSourceType::S3_Plain;
} }
/// Notes:
/// - supports BACKUP to this disk
/// - does not support INSERT into MergeTree table on this disk
bool isWriteOnce() const override { return true; }
}; };
} }

View File

@ -22,6 +22,7 @@
#include <IO/S3Common.h> #include <IO/S3Common.h>
#include <Storages/StorageS3Settings.h> #include <Storages/StorageS3Settings.h>
#include <Core/ServerUUID.h>
namespace DB namespace DB
@ -30,90 +31,78 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
extern const int PATH_ACCESS_DENIED; extern const int LOGICAL_ERROR;
} }
namespace namespace
{ {
void checkWriteAccess(IDisk & disk) class CheckAccess
{ {
auto file = disk.writeFile("test_acl", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); public:
try static bool checkBatchRemove(S3ObjectStorage & storage, const String & key_with_trailing_slash)
{
file->write("test", 4);
}
catch (...)
{
/// Log current exception, because finalize() can throw a different exception.
tryLogCurrentException(__PRETTY_FUNCTION__);
file->finalize();
throw;
}
}
void checkReadAccess(const String & disk_name, IDisk & disk)
{
auto file = disk.readFile("test_acl");
String buf(4, '0');
file->readStrict(buf.data(), 4);
if (buf != "test")
throw Exception("No read access to S3 bucket in disk " + disk_name, ErrorCodes::PATH_ACCESS_DENIED);
}
void checkRemoveAccess(IDisk & disk)
{
disk.removeFile("test_acl");
}
bool checkBatchRemoveIsMissing(S3ObjectStorage & storage, const String & key_with_trailing_slash)
{
StoredObject object(key_with_trailing_slash + "_test_remove_objects_capability");
try
{
auto file = storage.writeObject(object, WriteMode::Rewrite);
file->write("test", 4);
file->finalize();
}
catch (...)
{ {
/// NOTE: key_with_trailing_slash is the disk prefix, it is required
/// because access is done via S3ObjectStorage not via IDisk interface
/// (since we don't have disk yet).
const String path = fmt::format("{}clickhouse_remove_objects_capability_{}", key_with_trailing_slash, getServerUUID());
StoredObject object(path);
try try
{ {
storage.removeObject(object); auto file = storage.writeObject(object, WriteMode::Rewrite);
file->write("test", 4);
file->finalize();
} }
catch (...) catch (...)
{ {
try
{
storage.removeObject(object);
}
catch (...)
{
}
return true; /// We don't have write access, therefore no information about batch remove.
} }
return false; /// We don't have write access, therefore no information about batch remove.
}
try
{
/// Uses `DeleteObjects` request (batch delete).
storage.removeObjects({object});
return false;
}
catch (const Exception &)
{
try try
{ {
storage.removeObject(object); /// Uses `DeleteObjects` request (batch delete).
storage.removeObjects({object});
return true;
} }
catch (...) catch (const Exception &)
{ {
try
{
storage.removeObject(object);
}
catch (...)
{
}
return false;
} }
return true;
} }
}
private:
static String getServerUUID()
{
DB::UUID server_uuid = DB::ServerUUID::get();
if (server_uuid == DB::UUIDHelpers::Nil)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Server UUID is not initialized");
return DB::toString(server_uuid);
}
};
} }
void registerDiskS3(DiskFactory & factory) void registerDiskS3(DiskFactory & factory, bool global_skip_access_check)
{ {
auto creator = [](const String & name, auto creator = [global_skip_access_check](
const Poco::Util::AbstractConfiguration & config, const String & name,
const String & config_prefix, const Poco::Util::AbstractConfiguration & config,
ContextPtr context, const String & config_prefix,
const DisksMap & /*map*/) -> DiskPtr ContextPtr context,
const DisksMap & /*map*/) -> DiskPtr
{ {
S3::URI uri(Poco::URI(config.getString(config_prefix + ".endpoint"))); S3::URI uri(Poco::URI(config.getString(config_prefix + ".endpoint")));
@ -144,12 +133,12 @@ void registerDiskS3(DiskFactory & factory)
metadata_storage = std::make_shared<MetadataStorageFromDisk>(metadata_disk, uri.key); metadata_storage = std::make_shared<MetadataStorageFromDisk>(metadata_disk, uri.key);
} }
bool skip_access_check = config.getBool(config_prefix + ".skip_access_check", false); /// NOTE: should we still perform this check for clickhouse-disks?
bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false);
if (!skip_access_check) if (!skip_access_check)
{ {
/// If `support_batch_delete` is turned on (default), check and possibly switch it off. /// If `support_batch_delete` is turned on (default), check and possibly switch it off.
if (s3_capabilities.support_batch_delete && checkBatchRemoveIsMissing(*s3_storage, uri.key)) if (s3_capabilities.support_batch_delete && !CheckAccess::checkBatchRemove(*s3_storage, uri.key))
{ {
LOG_WARNING( LOG_WARNING(
&Poco::Logger::get("registerDiskS3"), &Poco::Logger::get("registerDiskS3"),
@ -165,7 +154,7 @@ void registerDiskS3(DiskFactory & factory)
bool send_metadata = config.getBool(config_prefix + ".send_metadata", false); bool send_metadata = config.getBool(config_prefix + ".send_metadata", false);
uint64_t copy_thread_pool_size = config.getUInt(config_prefix + ".thread_pool_size", 16); uint64_t copy_thread_pool_size = config.getUInt(config_prefix + ".thread_pool_size", 16);
std::shared_ptr<DiskObjectStorage> s3disk = std::make_shared<DiskObjectStorage>( DiskObjectStoragePtr s3disk = std::make_shared<DiskObjectStorage>(
name, name,
uri.key, uri.key,
type == "s3" ? "DiskS3" : "DiskS3Plain", type == "s3" ? "DiskS3" : "DiskS3Plain",
@ -174,15 +163,7 @@ void registerDiskS3(DiskFactory & factory)
send_metadata, send_metadata,
copy_thread_pool_size); copy_thread_pool_size);
/// This code is used only to check access to the corresponding disk. s3disk->startup(context, skip_access_check);
if (!skip_access_check)
{
checkWriteAccess(*s3disk);
checkReadAccess(name, *s3disk);
checkRemoveAccess(*s3disk);
}
s3disk->startup(context);
std::shared_ptr<IDisk> disk_result = s3disk; std::shared_ptr<IDisk> disk_result = s3disk;
@ -196,6 +177,6 @@ void registerDiskS3(DiskFactory & factory)
#else #else
void registerDiskS3(DiskFactory &) {} void registerDiskS3(DiskFactory &, bool /* global_skip_access_check */) {}
#endif #endif

View File

@ -13,7 +13,6 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int FILE_DOESNT_EXIST; extern const int FILE_DOESNT_EXIST;
extern const int NETWORK_ERROR;
} }
MetadataStorageFromStaticFilesWebServer::MetadataStorageFromStaticFilesWebServer( MetadataStorageFromStaticFilesWebServer::MetadataStorageFromStaticFilesWebServer(
@ -38,7 +37,7 @@ bool MetadataStorageFromStaticFilesWebServer::exists(const std::string & path) c
if (fs_path.has_extension()) if (fs_path.has_extension())
fs_path = fs_path.parent_path(); fs_path = fs_path.parent_path();
initializeIfNeeded(fs_path, false); initializeIfNeeded(fs_path);
if (object_storage.files.empty()) if (object_storage.files.empty())
return false; return false;
@ -123,39 +122,21 @@ std::vector<std::string> MetadataStorageFromStaticFilesWebServer::listDirectory(
return result; return result;
} }
bool MetadataStorageFromStaticFilesWebServer::initializeIfNeeded(const std::string & path, std::optional<bool> throw_on_error) const void MetadataStorageFromStaticFilesWebServer::initializeIfNeeded(const std::string & path) const
{ {
if (object_storage.files.find(path) == object_storage.files.end()) if (object_storage.files.find(path) == object_storage.files.end())
{ {
try object_storage.initialize(fs::path(object_storage.url) / path);
{
object_storage.initialize(fs::path(object_storage.url) / path);
}
catch (...)
{
const auto message = getCurrentExceptionMessage(false);
bool can_throw = throw_on_error.has_value() ? *throw_on_error : CurrentThread::isInitialized() && CurrentThread::get().getQueryContext();
if (can_throw)
throw Exception(ErrorCodes::NETWORK_ERROR, "Cannot load disk metadata. Error: {}", message);
LOG_TRACE(&Poco::Logger::get("DiskWeb"), "Cannot load disk metadata. Error: {}", message);
return false;
}
} }
return true;
} }
DirectoryIteratorPtr MetadataStorageFromStaticFilesWebServer::iterateDirectory(const std::string & path) const DirectoryIteratorPtr MetadataStorageFromStaticFilesWebServer::iterateDirectory(const std::string & path) const
{ {
std::vector<fs::path> dir_file_paths; std::vector<fs::path> dir_file_paths;
if (!initializeIfNeeded(path)) initializeIfNeeded(path);
{ if (!exists(path))
return std::make_unique<StaticDirectoryIterator>(std::move(dir_file_paths)); return std::make_unique<StaticDirectoryIterator>(std::move(dir_file_paths));
}
assertExists(path);
for (const auto & [file_path, _] : object_storage.files) for (const auto & [file_path, _] : object_storage.files)
{ {

View File

@ -19,7 +19,7 @@ private:
void assertExists(const std::string & path) const; void assertExists(const std::string & path) const;
bool initializeIfNeeded(const std::string & path, std::optional<bool> throw_on_error = std::nullopt) const; void initializeIfNeeded(const std::string & path) const;
public: public:
explicit MetadataStorageFromStaticFilesWebServer(const WebObjectStorage & object_storage_); explicit MetadataStorageFromStaticFilesWebServer(const WebObjectStorage & object_storage_);

View File

@ -46,7 +46,10 @@ void WebObjectStorage::initialize(const String & uri_path) const
Poco::Net::HTTPRequest::HTTP_GET, Poco::Net::HTTPRequest::HTTP_GET,
ReadWriteBufferFromHTTP::OutStreamCallback(), ReadWriteBufferFromHTTP::OutStreamCallback(),
ConnectionTimeouts::getHTTPTimeouts(getContext()), ConnectionTimeouts::getHTTPTimeouts(getContext()),
credentials); credentials,
/* max_redirects= */ 0,
/* buffer_size_= */ DBMS_DEFAULT_BUFFER_SIZE,
getContext()->getReadSettings());
String file_name; String file_name;
FileData file_data{}; FileData file_data{};
@ -82,6 +85,15 @@ void WebObjectStorage::initialize(const String & uri_path) const
files.emplace(std::make_pair(dir_name, FileData({ .type = FileType::Directory }))); files.emplace(std::make_pair(dir_name, FileData({ .type = FileType::Directory })));
} }
catch (HTTPException & e)
{
/// 404 - no files
if (e.getHTTPStatus() == Poco::Net::HTTPResponse::HTTP_NOT_FOUND)
return;
e.addMessage("while loading disk metadata");
throw;
}
catch (Exception & e) catch (Exception & e)
{ {
e.addMessage("while loading disk metadata"); e.addMessage("while loading disk metadata");

View File

@ -14,15 +14,17 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
} }
void registerDiskWebServer(DiskFactory & factory) void registerDiskWebServer(DiskFactory & factory, bool global_skip_access_check)
{ {
auto creator = [](const String & disk_name, auto creator = [global_skip_access_check](
const Poco::Util::AbstractConfiguration & config, const String & disk_name,
const String & config_prefix, const Poco::Util::AbstractConfiguration & config,
ContextPtr context, const String & config_prefix,
const DisksMap & /*map*/) -> DiskPtr ContextPtr context,
const DisksMap & /*map*/) -> DiskPtr
{ {
String uri{config.getString(config_prefix + ".endpoint")}; String uri{config.getString(config_prefix + ".endpoint")};
bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false);
if (!uri.ends_with('/')) if (!uri.ends_with('/'))
throw Exception( throw Exception(
@ -41,7 +43,7 @@ void registerDiskWebServer(DiskFactory & factory)
auto metadata_storage = std::make_shared<MetadataStorageFromStaticFilesWebServer>(assert_cast<const WebObjectStorage &>(*object_storage)); auto metadata_storage = std::make_shared<MetadataStorageFromStaticFilesWebServer>(assert_cast<const WebObjectStorage &>(*object_storage));
std::string root_path; std::string root_path;
return std::make_shared<DiskObjectStorage>( DiskPtr disk = std::make_shared<DiskObjectStorage>(
disk_name, disk_name,
root_path, root_path,
"DiskWebServer", "DiskWebServer",
@ -49,6 +51,8 @@ void registerDiskWebServer(DiskFactory & factory)
object_storage, object_storage,
/* send_metadata */false, /* send_metadata */false,
/* threadpool_size */16); /* threadpool_size */16);
disk->startup(context, skip_access_check);
return disk;
}; };
factory.registerDiskType("web", creator); factory.registerDiskType("web", creator);

View File

@ -7,55 +7,55 @@
namespace DB namespace DB
{ {
void registerDiskLocal(DiskFactory & factory); void registerDiskLocal(DiskFactory & factory, bool global_skip_access_check);
void registerDiskMemory(DiskFactory & factory); void registerDiskMemory(DiskFactory & factory, bool global_skip_access_check);
#if USE_AWS_S3 #if USE_AWS_S3
void registerDiskS3(DiskFactory & factory); void registerDiskS3(DiskFactory & factory, bool global_skip_access_check);
#endif #endif
#if USE_AZURE_BLOB_STORAGE #if USE_AZURE_BLOB_STORAGE
void registerDiskAzureBlobStorage(DiskFactory & factory); void registerDiskAzureBlobStorage(DiskFactory & factory, bool global_skip_access_check);
#endif #endif
#if USE_SSL #if USE_SSL
void registerDiskEncrypted(DiskFactory & factory); void registerDiskEncrypted(DiskFactory & factory, bool global_skip_access_check);
#endif #endif
#if USE_HDFS #if USE_HDFS
void registerDiskHDFS(DiskFactory & factory); void registerDiskHDFS(DiskFactory & factory, bool global_skip_access_check);
#endif #endif
void registerDiskWebServer(DiskFactory & factory); void registerDiskWebServer(DiskFactory & factory, bool global_skip_access_check);
void registerDiskCache(DiskFactory & factory); void registerDiskCache(DiskFactory & factory, bool global_skip_access_check);
void registerDisks() void registerDisks(bool global_skip_access_check)
{ {
auto & factory = DiskFactory::instance(); auto & factory = DiskFactory::instance();
registerDiskLocal(factory); registerDiskLocal(factory, global_skip_access_check);
registerDiskMemory(factory); registerDiskMemory(factory, global_skip_access_check);
#if USE_AWS_S3 #if USE_AWS_S3
registerDiskS3(factory); registerDiskS3(factory, global_skip_access_check);
#endif #endif
#if USE_AZURE_BLOB_STORAGE #if USE_AZURE_BLOB_STORAGE
registerDiskAzureBlobStorage(factory); registerDiskAzureBlobStorage(factory, global_skip_access_check);
#endif #endif
#if USE_SSL #if USE_SSL
registerDiskEncrypted(factory); registerDiskEncrypted(factory, global_skip_access_check);
#endif #endif
#if USE_HDFS #if USE_HDFS
registerDiskHDFS(factory); registerDiskHDFS(factory, global_skip_access_check);
#endif #endif
registerDiskWebServer(factory); registerDiskWebServer(factory, global_skip_access_check);
registerDiskCache(factory); registerDiskCache(factory, global_skip_access_check);
} }
} }

View File

@ -2,5 +2,10 @@
namespace DB namespace DB
{ {
void registerDisks();
/// @param global_skip_access_check - skip access check regardless regardless
/// .skip_access_check config directive (used
/// for clickhouse-disks)
void registerDisks(bool global_skip_access_check);
} }

View File

@ -312,15 +312,29 @@ void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPR
|| status == Poco::Net::HTTPResponse::HTTP_PARTIAL_CONTENT /// Reading with Range header was successful. || status == Poco::Net::HTTPResponse::HTTP_PARTIAL_CONTENT /// Reading with Range header was successful.
|| (isRedirect(status) && allow_redirects))) || (isRedirect(status) && allow_redirects)))
{ {
std::stringstream error_message; // STYLE_CHECK_ALLOW_STD_STRING_STREAM int code = status == Poco::Net::HTTPResponse::HTTP_TOO_MANY_REQUESTS
error_message.exceptions(std::ios::failbit); ? ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS
error_message << "Received error from remote server " << request.getURI() << ". HTTP status code: " << status << " " : ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER;
<< response.getReason() << ", body: " << istr.rdbuf();
throw Exception(error_message.str(), std::stringstream body; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
status == HTTP_TOO_MANY_REQUESTS ? ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS body.exceptions(std::ios::failbit);
: ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER); body << istr.rdbuf();
throw HTTPException(code, request.getURI(), status, response.getReason(), body.str());
} }
} }
std::string HTTPException::makeExceptionMessage(
const std::string & uri,
Poco::Net::HTTPResponse::HTTPStatus http_status,
const std::string & reason,
const std::string & body)
{
return fmt::format(
"Received error from remote server {}. "
"HTTP status code: {} {}, "
"body: {}",
uri, http_status, reason, body);
}
} }

View File

@ -17,8 +17,6 @@
namespace DB namespace DB
{ {
constexpr int HTTP_TOO_MANY_REQUESTS = 429;
class HTTPServerResponse; class HTTPServerResponse;
class SingleEndpointHTTPSessionPool : public PoolBase<Poco::Net::HTTPClientSession> class SingleEndpointHTTPSessionPool : public PoolBase<Poco::Net::HTTPClientSession>
@ -35,6 +33,38 @@ public:
SingleEndpointHTTPSessionPool(const std::string & host_, UInt16 port_, bool https_, size_t max_pool_size_); SingleEndpointHTTPSessionPool(const std::string & host_, UInt16 port_, bool https_, size_t max_pool_size_);
}; };
class HTTPException : public Exception
{
public:
HTTPException(
int code,
const std::string & uri,
Poco::Net::HTTPResponse::HTTPStatus http_status_,
const std::string & reason,
const std::string & body
)
: Exception(makeExceptionMessage(uri, http_status_, reason, body), code)
, http_status(http_status_)
{}
HTTPException * clone() const override { return new HTTPException(*this); }
void rethrow() const override { throw *this; }
int getHTTPStatus() const { return http_status; }
private:
Poco::Net::HTTPResponse::HTTPStatus http_status{};
static std::string makeExceptionMessage(
const std::string & uri,
Poco::Net::HTTPResponse::HTTPStatus http_status,
const std::string & reason,
const std::string & body);
const char * name() const noexcept override { return "DB::HTTPException"; }
const char * className() const noexcept override { return "DB::HTTPException"; }
};
using PooledHTTPSessionPtr = SingleEndpointHTTPSessionPool::Entry; using PooledHTTPSessionPtr = SingleEndpointHTTPSessionPool::Entry;
using HTTPSessionPtr = std::shared_ptr<Poco::Net::HTTPClientSession>; using HTTPSessionPtr = std::shared_ptr<Poco::Net::HTTPClientSession>;

View File

@ -1829,9 +1829,22 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1)) if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1))
{ {
Names columns_for_sampling = metadata_snapshot->getColumnsRequiredForSampling(); // we evaluate sampling for Merge lazily so we need to get all the columns
additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), if (storage->getName() == "Merge")
columns_for_sampling.begin(), columns_for_sampling.end()); {
const auto columns = metadata_snapshot->getColumns().getAll();
for (const auto & column : columns)
{
additional_required_columns_after_prewhere.push_back(column.name);
}
}
else
{
Names columns_for_sampling = metadata_snapshot->getColumnsRequiredForSampling();
additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(),
columns_for_sampling.begin(), columns_for_sampling.end());
}
} }
if (storage && query.final()) if (storage && query.final())

View File

@ -33,7 +33,7 @@ try
registerTableFunctions(); registerTableFunctions();
registerStorages(); registerStorages();
registerDictionaries(); registerDictionaries();
registerDisks(); registerDisks(/* global_skip_access_check= */ true);
registerFormats(); registerFormats();
return true; return true;

View File

@ -1094,7 +1094,153 @@ bool ParserCollectionOfLiterals<Collection>::parseImpl(Pos & pos, ASTPtr & node,
template bool ParserCollectionOfLiterals<Array>::parseImpl(Pos & pos, ASTPtr & node, Expected & expected); template bool ParserCollectionOfLiterals<Array>::parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
template bool ParserCollectionOfLiterals<Tuple>::parseImpl(Pos & pos, ASTPtr & node, Expected & expected); template bool ParserCollectionOfLiterals<Tuple>::parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
template bool ParserCollectionOfLiterals<Map>::parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
namespace
{
class ICollection;
using Collections = std::vector<std::unique_ptr<ICollection>>;
class ICollection
{
public:
virtual ~ICollection() = default;
virtual bool parse(IParser::Pos & pos, Collections & collections, ASTPtr & node, Expected & expected) = 0;
};
template <class Container, TokenType end_token>
class CommonCollection : public ICollection
{
public:
bool parse(IParser::Pos & pos, Collections & collections, ASTPtr & node, Expected & expected) override;
private:
Container container;
};
class MapCollection : public ICollection
{
public:
bool parse(IParser::Pos & pos, Collections & collections, ASTPtr & node, Expected & expected) override;
private:
Map container;
};
bool parseAllCollectionsStart(IParser::Pos & pos, Collections & collections, Expected & /*expected*/)
{
if (pos->type == TokenType::OpeningCurlyBrace)
collections.push_back(std::make_unique<MapCollection>());
else if (pos->type == TokenType::OpeningRoundBracket)
collections.push_back(std::make_unique<CommonCollection<Tuple, TokenType::ClosingRoundBracket>>());
else if (pos->type == TokenType::OpeningSquareBracket)
collections.push_back(std::make_unique<CommonCollection<Array, TokenType::ClosingSquareBracket>>());
else
return false;
++pos;
return true;
}
template <class Container, TokenType end_token>
bool CommonCollection<Container, end_token>::parse(IParser::Pos & pos, Collections & collections, ASTPtr & node, Expected & expected)
{
if (node)
{
container.push_back(std::move(node->as<ASTLiteral &>().value));
node.reset();
}
ASTPtr literal;
ParserLiteral literal_p;
ParserToken comma_p(TokenType::Comma);
ParserToken end_p(end_token);
while (true)
{
if (end_p.ignore(pos, expected))
{
node = std::make_shared<ASTLiteral>(std::move(container));
break;
}
if (!container.empty() && !comma_p.ignore(pos, expected))
return false;
if (literal_p.parse(pos, literal, expected))
container.push_back(std::move(literal->as<ASTLiteral &>().value));
else
return parseAllCollectionsStart(pos, collections, expected);
}
return true;
}
bool MapCollection::parse(IParser::Pos & pos, Collections & collections, ASTPtr & node, Expected & expected)
{
if (node)
{
container.push_back(std::move(node->as<ASTLiteral &>().value));
node.reset();
}
ASTPtr literal;
ParserLiteral literal_p;
ParserToken comma_p(TokenType::Comma);
ParserToken colon_p(TokenType::Colon);
ParserToken end_p(TokenType::ClosingCurlyBrace);
while (true)
{
if (end_p.ignore(pos, expected))
{
node = std::make_shared<ASTLiteral>(std::move(container));
break;
}
if (!container.empty() && !comma_p.ignore(pos, expected))
return false;
if (!literal_p.parse(pos, literal, expected))
return false;
if (!colon_p.parse(pos, literal, expected))
return false;
container.push_back(std::move(literal->as<ASTLiteral &>().value));
if (literal_p.parse(pos, literal, expected))
container.push_back(std::move(literal->as<ASTLiteral &>().value));
else
return parseAllCollectionsStart(pos, collections, expected);
}
return true;
}
}
bool ParserAllCollectionsOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Collections collections;
if (!parseAllCollectionsStart(pos, collections, expected))
return false;
while (!collections.empty())
{
if (!collections.back()->parse(pos, collections, node, expected))
return false;
if (node)
collections.pop_back();
}
return true;
}
bool ParserLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool ParserLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{ {

View File

@ -301,6 +301,17 @@ protected:
} }
}; };
/** Parses all collections of literals and their various combinations
* Used in parsing parameters for SET query
*/
class ParserAllCollectionsOfLiterals : public IParserBase
{
public:
protected:
const char * getName() const override { return "combination of maps, arrays, tuples"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** The literal is one of: NULL, UInt64, Int64, Float64, String. /** The literal is one of: NULL, UInt64, Int64, Float64, String.
*/ */

View File

@ -925,7 +925,7 @@ public:
, ErrorCodes::SYNTAX_ERROR); , ErrorCodes::SYNTAX_ERROR);
} }
if (allow_function_parameters && ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)) if (allow_function_parameters && !parameters && ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected))
{ {
parameters = std::make_shared<ASTExpressionList>(); parameters = std::make_shared<ASTExpressionList>();
std::swap(parameters->children, elements); std::swap(parameters->children, elements);

View File

@ -4,13 +4,15 @@
#include <Parsers/CommonParsers.h> #include <Parsers/CommonParsers.h>
#include <Parsers/ParserSetQuery.h> #include <Parsers/ParserSetQuery.h>
#include <Parsers/ExpressionListParsers.h> #include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/SettingValueFromAST.h> #include <Parsers/SettingValueFromAST.h>
#include <Core/Names.h> #include <Core/Names.h>
#include <IO/ReadBufferFromString.h> #include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromString.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/Operators.h>
#include <Common/FieldVisitorToString.h> #include <Common/FieldVisitorToString.h>
#include <Common/SettingsChanges.h> #include <Common/SettingsChanges.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
@ -23,21 +25,75 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
} }
static NameToNameMap::value_type convertToQueryParameter(SettingChange change)
{
auto name = change.getName().substr(strlen(QUERY_PARAMETER_NAME_PREFIX));
if (name.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter name cannot be empty");
auto value = applyVisitor(FieldVisitorToString(), change.getFieldValue()); class ParameterFieldVisitorToString : public StaticVisitor<String>
/// writeQuoted is not always quoted in line with SQL standard https://github.com/ClickHouse/ClickHouse/blob/master/src/IO/WriteHelpers.h {
if (value.starts_with('\'')) public:
template <class T>
String operator() (const T & x) const
{ {
ReadBufferFromOwnString buf(value); FieldVisitorToString visitor;
readQuoted(value, buf); return visitor(x);
} }
return {name, value};
} String operator() (const Array & x) const
{
WriteBufferFromOwnString wb;
wb << '[';
for (Array::const_iterator it = x.begin(); it != x.end(); ++it)
{
if (it != x.begin())
wb.write(", ", 2);
wb << applyVisitor(*this, *it);
}
wb << ']';
return wb.str();
}
String operator() (const Map & x) const
{
WriteBufferFromOwnString wb;
wb << '{';
auto it = x.begin();
while (it != x.end())
{
if (it != x.begin())
wb << ", ";
wb << applyVisitor(*this, *it);
++it;
if (it != x.end())
{
wb << ':';
wb << applyVisitor(*this, *it);
++it;
}
}
wb << '}';
return wb.str();
}
String operator() (const Tuple & x) const
{
WriteBufferFromOwnString wb;
wb << '(';
for (auto it = x.begin(); it != x.end(); ++it)
{
if (it != x.begin())
wb << ", ";
wb << applyVisitor(*this, *it);
}
wb << ')';
return wb.str();
}
};
class ParserLiteralOrMap : public IParserBase class ParserLiteralOrMap : public IParserBase
@ -92,6 +148,48 @@ protected:
} }
}; };
/// Parse Identifier, Literal, Array/Tuple/Map of literals
bool parseParameterValueIntoString(IParser::Pos & pos, String & value, Expected & expected)
{
ASTPtr node;
/// 1. Identifier
ParserCompoundIdentifier identifier_p;
if (identifier_p.parse(pos, node, expected))
{
tryGetIdentifierNameInto(node, value);
return true;
}
/// 2. Literal
ParserLiteral literal_p;
if (literal_p.parse(pos, node, expected))
{
value = applyVisitor(FieldVisitorToString(), node->as<ASTLiteral>()->value);
/// writeQuoted is not always quoted in line with SQL standard https://github.com/ClickHouse/ClickHouse/blob/master/src/IO/WriteHelpers.h
if (value.starts_with('\''))
{
ReadBufferFromOwnString buf(value);
readQuoted(value, buf);
}
return true;
}
/// 3. Map, Array, Tuple of literals and their combination
ParserAllCollectionsOfLiterals all_collections_p;
if (all_collections_p.parse(pos, node, expected))
{
value = applyVisitor(ParameterFieldVisitorToString(), node->as<ASTLiteral>()->value);
return true;
}
return false;
}
/// Parse `name = value`. /// Parse `name = value`.
bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & pos, Expected & expected) bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & pos, Expected & expected)
{ {
@ -132,31 +230,55 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p
return true; return true;
} }
bool ParserSetQuery::parseNameValuePairWithDefault(SettingChange & change, String & default_settings, IParser::Pos & pos, Expected & expected) bool ParserSetQuery::parseNameValuePairWithParameterOrDefault(
SettingChange & change, String & default_settings, ParserSetQuery::Parameter & parameter, IParser::Pos & pos, Expected & expected)
{ {
ParserCompoundIdentifier name_p; ParserCompoundIdentifier name_p;
ParserLiteralOrMap value_p; ParserLiteralOrMap value_p;
ParserToken s_eq(TokenType::Equals); ParserToken s_eq(TokenType::Equals);
ParserFunction function_p; ParserFunction function_p;
ASTPtr name; ASTPtr node;
ASTPtr value; String name;
bool is_default = false;
ASTPtr function_ast; ASTPtr function_ast;
if (!name_p.parse(pos, name, expected)) if (!name_p.parse(pos, node, expected))
return false; return false;
if (!s_eq.ignore(pos, expected)) if (!s_eq.ignore(pos, expected))
return false; return false;
tryGetIdentifierNameInto(node, name);
/// Parameter
if (name.starts_with(QUERY_PARAMETER_NAME_PREFIX))
{
name = name.substr(strlen(QUERY_PARAMETER_NAME_PREFIX));
if (name.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter name cannot be empty");
String value;
if (!parseParameterValueIntoString(pos, value, expected))
return false;
parameter = {std::move(name), std::move(value)};
return true;
}
/// Default
if (ParserKeyword("DEFAULT").ignore(pos, expected))
{
default_settings = name;
return true;
}
/// Setting
if (ParserKeyword("TRUE").ignore(pos, expected)) if (ParserKeyword("TRUE").ignore(pos, expected))
value = std::make_shared<ASTLiteral>(Field(static_cast<UInt64>(1))); node = std::make_shared<ASTLiteral>(Field(static_cast<UInt64>(1)));
else if (ParserKeyword("FALSE").ignore(pos, expected)) else if (ParserKeyword("FALSE").ignore(pos, expected))
value = std::make_shared<ASTLiteral>(Field(static_cast<UInt64>(0))); node = std::make_shared<ASTLiteral>(Field(static_cast<UInt64>(0)));
else if (ParserKeyword("DEFAULT").ignore(pos, expected))
is_default = true;
/// for SETTINGS disk=disk(type='s3', path='', ...)
else if (function_p.parse(pos, function_ast, expected) && function_ast->as<ASTFunction>()->name == "disk") else if (function_p.parse(pos, function_ast, expected) && function_ast->as<ASTFunction>()->name == "disk")
{ {
tryGetIdentifierNameInto(name, change.getName()); tryGetIdentifierNameInto(name, change.getName());
@ -164,14 +286,11 @@ bool ParserSetQuery::parseNameValuePairWithDefault(SettingChange & change, Strin
return true; return true;
} }
else if (!value_p.parse(pos, value, expected)) else if (!value_p.parse(pos, node, expected))
return false; return false;
tryGetIdentifierNameInto(name, change.getName()); change.name = name;
if (is_default) change.value = node->as<ASTLiteral &>().value;
default_settings = change.getName();
else
change.setValue(value->as<ASTLiteral &>().value);
return true; return true;
} }
@ -202,19 +321,19 @@ bool ParserSetQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if ((!changes.empty() || !query_parameters.empty() || !default_settings.empty()) && !s_comma.ignore(pos)) if ((!changes.empty() || !query_parameters.empty() || !default_settings.empty()) && !s_comma.ignore(pos))
break; break;
/// Either a setting or a parameter for prepared statement (if name starts with QUERY_PARAMETER_NAME_PREFIX) SettingChange setting;
SettingChange current;
String name_of_default_setting; String name_of_default_setting;
Parameter parameter;
if (!parseNameValuePairWithDefault(current, name_of_default_setting, pos, expected)) if (!parseNameValuePairWithParameterOrDefault(setting, name_of_default_setting, parameter, pos, expected))
return false; return false;
if (current.getName().starts_with(QUERY_PARAMETER_NAME_PREFIX)) if (!parameter.first.empty())
query_parameters.emplace(convertToQueryParameter(std::move(current))); query_parameters.emplace(std::move(parameter));
else if (!name_of_default_setting.empty()) else if (!name_of_default_setting.empty())
default_settings.emplace_back(std::move(name_of_default_setting)); default_settings.emplace_back(std::move(name_of_default_setting));
else else
changes.push_back(std::move(current)); changes.push_back(std::move(setting));
} }
auto query = std::make_shared<ASTSetQuery>(); auto query = std::make_shared<ASTSetQuery>();

View File

@ -15,9 +15,18 @@ class SettingChange;
class ParserSetQuery : public IParserBase class ParserSetQuery : public IParserBase
{ {
public: public:
using Parameter = std::pair<std::string, std::string>;
explicit ParserSetQuery(bool parse_only_internals_ = false) : parse_only_internals(parse_only_internals_) {} explicit ParserSetQuery(bool parse_only_internals_ = false) : parse_only_internals(parse_only_internals_) {}
static bool parseNameValuePair(SettingChange & change, IParser::Pos & pos, Expected & expected); static bool parseNameValuePair(SettingChange & change, IParser::Pos & pos, Expected & expected);
static bool parseNameValuePairWithDefault(SettingChange & change, String & default_settings, IParser::Pos & pos, Expected & expected);
static bool parseNameValuePairWithParameterOrDefault(SettingChange & change,
String & default_settings,
Parameter & parameter,
IParser::Pos & pos,
Expected & expected);
protected: protected:
const char * getName() const override { return "SET query"; } const char * getName() const override { return "SET query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;

View File

@ -22,7 +22,7 @@ bool ParserTableExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
auto res = std::make_shared<ASTTableExpression>(); auto res = std::make_shared<ASTTableExpression>();
if (!ParserWithOptionalAlias(std::make_unique<ParserSubquery>(), true).parse(pos, res->subquery, expected) if (!ParserWithOptionalAlias(std::make_unique<ParserSubquery>(), true).parse(pos, res->subquery, expected)
&& !ParserWithOptionalAlias(std::make_unique<ParserFunction>(true, true), true).parse(pos, res->table_function, expected) && !ParserWithOptionalAlias(std::make_unique<ParserFunction>(false, true), true).parse(pos, res->table_function, expected)
&& !ParserWithOptionalAlias(std::make_unique<ParserCompoundIdentifier>(true, true), true) && !ParserWithOptionalAlias(std::make_unique<ParserCompoundIdentifier>(true, true), true)
.parse(pos, res->database_and_table_name, expected)) .parse(pos, res->database_and_table_name, expected))
return false; return false;

View File

@ -99,32 +99,24 @@ Pipe StorageHDFSCluster::read(
addColumnsStructureToQueryWithClusterEngine( addColumnsStructureToQueryWithClusterEngine(
query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), 3, getName()); query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), 3, getName());
for (const auto & replicas : cluster->getShardsAddresses()) const auto & current_settings = context->getSettingsRef();
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
for (const auto & shard_info : cluster->getShardsInfo())
{ {
/// There will be only one replica, because we consider each replica as a shard auto try_results = shard_info.pool->getMany(timeouts, &current_settings, PoolMode::GET_MANY);
for (const auto & node : replicas) for (auto & try_result : try_results)
{ {
auto connection = std::make_shared<Connection>(
node.host_name, node.port, context->getGlobalContext()->getCurrentDatabase(),
node.user, node.password, node.quota_key, node.cluster, node.cluster_secret,
"HDFSClusterInititiator",
node.compression,
node.secure
);
/// For unknown reason global context is passed to IStorage::read() method
/// So, task_identifier is passed as constructor argument. It is more obvious.
auto remote_query_executor = std::make_shared<RemoteQueryExecutor>( auto remote_query_executor = std::make_shared<RemoteQueryExecutor>(
connection, shard_info.pool,
queryToString(query_to_send), std::vector<IConnectionPool::Entry>{try_result},
header, queryToString(query_to_send),
context, header,
/*throttler=*/nullptr, context,
scalars, /*throttler=*/nullptr,
Tables(), scalars,
processed_stage, Tables(),
RemoteQueryExecutor::Extension{.task_iterator = callback}); processed_stage,
RemoteQueryExecutor::Extension{.task_iterator = callback});
pipes.emplace_back(std::make_shared<RemoteSource>(remote_query_executor, add_agg_info, false)); pipes.emplace_back(std::make_shared<RemoteSource>(remote_query_executor, add_agg_info, false));
} }

View File

@ -253,7 +253,7 @@ bool IStorage::isStaticStorage() const
if (storage_policy) if (storage_policy)
{ {
for (const auto & disk : storage_policy->getDisks()) for (const auto & disk : storage_policy->getDisks())
if (!disk->isReadOnly()) if (!(disk->isReadOnly() || disk->isWriteOnce()))
return false; return false;
return true; return true;
} }

View File

@ -583,7 +583,8 @@ public:
/// Returns storage policy if storage supports it. /// Returns storage policy if storage supports it.
virtual StoragePolicyPtr getStoragePolicy() const { return {}; } virtual StoragePolicyPtr getStoragePolicy() const { return {}; }
/// Returns true if all disks of storage are read-only. /// Returns true if all disks of storage are read-only or write-once.
/// NOTE: write-once also does not support INSERTs/merges/... for MergeTree
virtual bool isStaticStorage() const; virtual bool isStaticStorage() const;
virtual bool supportsSubsetOfColumns() const { return false; } virtual bool supportsSubsetOfColumns() const { return false; }

View File

@ -315,7 +315,9 @@ MergeTreeReadTaskColumns getReadTaskColumns(
/// 1. Columns for row level filter /// 1. Columns for row level filter
if (prewhere_info->row_level_filter) if (prewhere_info->row_level_filter)
{ {
Names row_filter_column_names = prewhere_info->row_level_filter->getRequiredColumnsNames(); Names row_filter_column_names = prewhere_info->row_level_filter->getRequiredColumnsNames();
injectRequiredColumns(
data_part_info_for_reader, storage_snapshot, with_subcolumns, row_filter_column_names);
result.pre_columns.push_back(storage_snapshot->getColumnsByNames(options, row_filter_column_names)); result.pre_columns.push_back(storage_snapshot->getColumnsByNames(options, row_filter_column_names));
pre_name_set.insert(row_filter_column_names.begin(), row_filter_column_names.end()); pre_name_set.insert(row_filter_column_names.begin(), row_filter_column_names.end());
} }
@ -323,7 +325,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(
/// 2. Columns for prewhere /// 2. Columns for prewhere
Names all_pre_column_names = prewhere_info->prewhere_actions->getRequiredColumnsNames(); Names all_pre_column_names = prewhere_info->prewhere_actions->getRequiredColumnsNames();
const auto injected_pre_columns = injectRequiredColumns( injectRequiredColumns(
data_part_info_for_reader, storage_snapshot, with_subcolumns, all_pre_column_names); data_part_info_for_reader, storage_snapshot, with_subcolumns, all_pre_column_names);
for (const auto & name : all_pre_column_names) for (const auto & name : all_pre_column_names)

View File

@ -314,18 +314,32 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge(
if (metadata_snapshot->hasAnyTTL() && merge_with_ttl_allowed && !ttl_merges_blocker.isCancelled()) if (metadata_snapshot->hasAnyTTL() && merge_with_ttl_allowed && !ttl_merges_blocker.isCancelled())
{ {
/// TTL delete is preferred to recompression /// TTL delete is preferred to recompression
TTLDeleteMergeSelector delete_ttl_selector( TTLDeleteMergeSelector drop_ttl_selector(
next_delete_ttl_merge_times_by_partition, next_delete_ttl_merge_times_by_partition,
current_time, current_time,
data_settings->merge_with_ttl_timeout, data_settings->merge_with_ttl_timeout,
data_settings->ttl_only_drop_parts); true);
parts_to_merge = delete_ttl_selector.select(parts_ranges, max_total_size_to_merge); /// The size of the completely expired part of TTL drop is not affected by the merge pressure and the size of the storage space
parts_to_merge = drop_ttl_selector.select(parts_ranges, data_settings->max_bytes_to_merge_at_max_space_in_pool);
if (!parts_to_merge.empty()) if (!parts_to_merge.empty())
{ {
future_part->merge_type = MergeType::TTLDelete; future_part->merge_type = MergeType::TTLDelete;
} }
else if (metadata_snapshot->hasAnyRecompressionTTL()) else if (!data_settings->ttl_only_drop_parts)
{
TTLDeleteMergeSelector delete_ttl_selector(
next_delete_ttl_merge_times_by_partition,
current_time,
data_settings->merge_with_ttl_timeout,
false);
parts_to_merge = delete_ttl_selector.select(parts_ranges, max_total_size_to_merge);
if (!parts_to_merge.empty())
future_part->merge_type = MergeType::TTLDelete;
}
if (parts_to_merge.empty() && metadata_snapshot->hasAnyRecompressionTTL())
{ {
TTLRecompressMergeSelector recompress_ttl_selector( TTLRecompressMergeSelector recompress_ttl_selector(
next_recompress_ttl_merge_times_by_partition, next_recompress_ttl_merge_times_by_partition,
@ -621,8 +635,16 @@ MergeTreeData::DataPartPtr MergeTreeDataMergerMutator::renameMergedTemporaryPart
size_t MergeTreeDataMergerMutator::estimateNeededDiskSpace(const MergeTreeData::DataPartsVector & source_parts) size_t MergeTreeDataMergerMutator::estimateNeededDiskSpace(const MergeTreeData::DataPartsVector & source_parts)
{ {
size_t res = 0; size_t res = 0;
time_t current_time = std::time(nullptr);
for (const MergeTreeData::DataPartPtr & part : source_parts) for (const MergeTreeData::DataPartPtr & part : source_parts)
{
/// Exclude expired parts
time_t part_max_ttl = part->ttl_infos.part_max_ttl;
if (part_max_ttl && part_max_ttl <= current_time)
continue;
res += part->getBytesOnDisk(); res += part->getBytesOnDisk();
}
return static_cast<size_t>(res * DISK_USAGE_COEFFICIENT_TO_RESERVE); return static_cast<size_t>(res * DISK_USAGE_COEFFICIENT_TO_RESERVE);
} }

View File

@ -1,7 +1,7 @@
#include "config.h" #include "config.h"
#if USE_AWS_S3 #if USE_AWS_S3
#include <Storages/StorageDelta.h> #include <Storages/StorageDeltaLake.h>
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <IO/ReadBufferFromS3.h> #include <IO/ReadBufferFromS3.h>
@ -47,7 +47,7 @@ void DeltaLakeMetadata::remove(const String & filename, uint64_t /*timestamp */)
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid table metadata, tried to remove {} before adding it", filename); throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid table metadata, tried to remove {} before adding it", filename);
} }
std::vector<String> DeltaLakeMetadata::ListCurrentFiles() && std::vector<String> DeltaLakeMetadata::listCurrentFiles() &&
{ {
std::vector<String> keys; std::vector<String> keys;
keys.reserve(file_update_time.size()); keys.reserve(file_update_time.size());
@ -61,10 +61,10 @@ std::vector<String> DeltaLakeMetadata::ListCurrentFiles() &&
JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, ContextPtr context) JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, ContextPtr context)
: base_configuration(configuration_), table_path(table_path_) : base_configuration(configuration_), table_path(table_path_)
{ {
Init(context); init(context);
} }
void JsonMetadataGetter::Init(ContextPtr context) void JsonMetadataGetter::init(ContextPtr context)
{ {
auto keys = getJsonLogFiles(); auto keys = getJsonLogFiles();
@ -180,7 +180,53 @@ void JsonMetadataGetter::handleJSON(const JSON & json)
} }
} }
StorageDelta::StorageDelta( namespace
{
StorageS3::S3Configuration getBaseConfiguration(const StorageS3Configuration & configuration)
{
return {configuration.url, configuration.auth_settings, configuration.request_settings, configuration.headers};
}
// DeltaLake stores data in parts in different files
// keys is vector of parts with latest version
// generateQueryFromKeys constructs query from parts filenames for
// underlying StorageS3 engine
String generateQueryFromKeys(const std::vector<String> & keys)
{
std::string new_query = fmt::format("{{{}}}", fmt::join(keys, ","));
return new_query;
}
StorageS3Configuration getAdjustedS3Configuration(
const ContextPtr & context,
StorageS3::S3Configuration & base_configuration,
const StorageS3Configuration & configuration,
const std::string & table_path,
Poco::Logger * log)
{
JsonMetadataGetter getter{base_configuration, table_path, context};
auto keys = getter.getFiles();
auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(keys);
LOG_DEBUG(log, "New uri: {}", new_uri);
LOG_DEBUG(log, "Table path: {}", table_path);
// set new url in configuration
StorageS3Configuration new_configuration;
new_configuration.url = new_uri;
new_configuration.auth_settings.access_key_id = configuration.auth_settings.access_key_id;
new_configuration.auth_settings.secret_access_key = configuration.auth_settings.secret_access_key;
new_configuration.format = configuration.format;
return new_configuration;
}
}
StorageDeltaLake::StorageDeltaLake(
const StorageS3Configuration & configuration_, const StorageS3Configuration & configuration_,
const StorageID & table_id_, const StorageID & table_id_,
ColumnsDescription columns_, ColumnsDescription columns_,
@ -189,28 +235,14 @@ StorageDelta::StorageDelta(
ContextPtr context_, ContextPtr context_,
std::optional<FormatSettings> format_settings_) std::optional<FormatSettings> format_settings_)
: IStorage(table_id_) : IStorage(table_id_)
, base_configuration{configuration_.url, configuration_.auth_settings, configuration_.request_settings, configuration_.headers} , base_configuration{getBaseConfiguration(configuration_)}
, log(&Poco::Logger::get("StorageDeltaLake (" + table_id_.table_name + ")")) , log(&Poco::Logger::get("StorageDeltaLake (" + table_id_.table_name + ")"))
, table_path(base_configuration.uri.key) , table_path(base_configuration.uri.key)
{ {
StorageInMemoryMetadata storage_metadata; StorageInMemoryMetadata storage_metadata;
StorageS3::updateS3Configuration(context_, base_configuration); StorageS3::updateS3Configuration(context_, base_configuration);
JsonMetadataGetter getter{base_configuration, table_path, context_}; auto new_configuration = getAdjustedS3Configuration(context_, base_configuration, configuration_, table_path, log);
auto keys = getter.getFiles();
auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys));
LOG_DEBUG(log, "New uri: {}", new_uri);
LOG_DEBUG(log, "Table path: {}", table_path);
// set new url in configuration
StorageS3Configuration new_configuration;
new_configuration.url = new_uri;
new_configuration.auth_settings.access_key_id = configuration_.auth_settings.access_key_id;
new_configuration.auth_settings.secret_access_key = configuration_.auth_settings.secret_access_key;
new_configuration.format = configuration_.format;
if (columns_.empty()) if (columns_.empty())
{ {
@ -238,7 +270,7 @@ StorageDelta::StorageDelta(
nullptr); nullptr);
} }
Pipe StorageDelta::read( Pipe StorageDeltaLake::read(
const Names & column_names, const Names & column_names,
const StorageSnapshotPtr & storage_snapshot, const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info, SelectQueryInfo & query_info,
@ -252,16 +284,18 @@ Pipe StorageDelta::read(
return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
} }
String StorageDelta::generateQueryFromKeys(std::vector<String> && keys) ColumnsDescription StorageDeltaLake::getTableStructureFromData(
const StorageS3Configuration & configuration, const std::optional<FormatSettings> & format_settings, ContextPtr ctx)
{ {
// DeltaLake store data parts in different files auto base_configuration = getBaseConfiguration(configuration);
// keys are filenames of parts StorageS3::updateS3Configuration(ctx, base_configuration);
// for StorageS3 to read all parts we need format {key1,key2,key3,...keyn} auto new_configuration = getAdjustedS3Configuration(
std::string new_query = fmt::format("{{{}}}", fmt::join(keys, ",")); ctx, base_configuration, configuration, base_configuration.uri.key, &Poco::Logger::get("StorageDeltaLake"));
return new_query; return StorageS3::getTableStructureFromData(
new_configuration, /*distributed processing*/ false, format_settings, ctx, /*object_infos*/ nullptr);
} }
void registerStorageDelta(StorageFactory & factory) void registerStorageDeltaLake(StorageFactory & factory)
{ {
factory.registerStorage( factory.registerStorage(
"DeltaLake", "DeltaLake",
@ -287,7 +321,7 @@ void registerStorageDelta(StorageFactory & factory)
configuration.format = "Parquet"; configuration.format = "Parquet";
} }
return std::make_shared<StorageDelta>( return std::make_shared<StorageDeltaLake>(
configuration, args.table_id, args.columns, args.constraints, args.comment, args.getContext(), std::nullopt); configuration, args.table_id, args.columns, args.constraints, args.comment, args.getContext(), std::nullopt);
}, },
{ {

View File

@ -32,7 +32,7 @@ public:
void setLastModifiedTime(const String & filename, uint64_t timestamp); void setLastModifiedTime(const String & filename, uint64_t timestamp);
void remove(const String & filename, uint64_t timestamp); void remove(const String & filename, uint64_t timestamp);
std::vector<String> ListCurrentFiles() &&; std::vector<String> listCurrentFiles() &&;
private: private:
std::unordered_map<String, uint64_t> file_update_time; std::unordered_map<String, uint64_t> file_update_time;
@ -44,10 +44,10 @@ class JsonMetadataGetter
public: public:
JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, ContextPtr context); JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, ContextPtr context);
std::vector<String> getFiles() { return std::move(metadata).ListCurrentFiles(); } std::vector<String> getFiles() { return std::move(metadata).listCurrentFiles(); }
private: private:
void Init(ContextPtr context); void init(ContextPtr context);
std::vector<String> getJsonLogFiles(); std::vector<String> getJsonLogFiles();
@ -60,13 +60,13 @@ private:
DeltaLakeMetadata metadata; DeltaLakeMetadata metadata;
}; };
class StorageDelta : public IStorage class StorageDeltaLake : public IStorage
{ {
public: public:
// 1. Parses internal file structure of table // 1. Parses internal file structure of table
// 2. Finds out parts with latest version // 2. Finds out parts with latest version
// 3. Creates url for underlying StorageS3 enigne to handle reads // 3. Creates url for underlying StorageS3 enigne to handle reads
StorageDelta( StorageDeltaLake(
const StorageS3Configuration & configuration_, const StorageS3Configuration & configuration_,
const StorageID & table_id_, const StorageID & table_id_,
ColumnsDescription columns_, ColumnsDescription columns_,
@ -87,14 +87,12 @@ public:
size_t max_block_size, size_t max_block_size,
size_t num_streams) override; size_t num_streams) override;
static ColumnsDescription getTableStructureFromData(
const StorageS3Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx);
private: private:
void Init(); void init();
// DeltaLake stores data in parts in different files
// keys is vector of parts with latest version
// generateQueryFromKeys constructs query from parts filenames for
// underlying StorageS3 engine
static String generateQueryFromKeys(std::vector<String> && keys);
StorageS3::S3Configuration base_configuration; StorageS3::S3Configuration base_configuration;
std::shared_ptr<StorageS3> s3engine; std::shared_ptr<StorageS3> s3engine;

View File

@ -28,115 +28,20 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
StorageHudi::StorageHudi( namespace
const StorageS3Configuration & configuration_,
const StorageID & table_id_,
ColumnsDescription columns_,
const ConstraintsDescription & constraints_,
const String & comment,
ContextPtr context_,
std::optional<FormatSettings> format_settings_)
: IStorage(table_id_)
, base_configuration{configuration_.url, configuration_.auth_settings, configuration_.request_settings, configuration_.headers}
, log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")"))
, table_path(base_configuration.uri.key)
{ {
StorageInMemoryMetadata storage_metadata;
StorageS3::updateS3Configuration(context_, base_configuration);
auto keys = getKeysFromS3(); StorageS3::S3Configuration getBaseConfiguration(const StorageS3Configuration & configuration)
auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(keys, configuration_.format); {
return {configuration.url, configuration.auth_settings, configuration.request_settings, configuration.headers};
LOG_DEBUG(log, "New uri: {}", new_uri);
LOG_DEBUG(log, "Table path: {}", table_path);
StorageS3Configuration new_configuration;
new_configuration.url = new_uri;
new_configuration.auth_settings.access_key_id = configuration_.auth_settings.access_key_id;
new_configuration.auth_settings.secret_access_key = configuration_.auth_settings.secret_access_key;
new_configuration.format = configuration_.format;
if (columns_.empty())
{
columns_ = StorageS3::getTableStructureFromData(
new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr);
storage_metadata.setColumns(columns_);
}
else
storage_metadata.setColumns(columns_);
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata);
s3engine = std::make_shared<StorageS3>(
new_configuration,
table_id_,
columns_,
constraints_,
comment,
context_,
format_settings_,
/* distributed_processing_ */ false,
nullptr);
} }
Pipe StorageHudi::read( /// Apache Hudi store parts of data in different files.
const Names & column_names, /// Every part file has timestamp in it.
const StorageSnapshotPtr & storage_snapshot, /// Every partition(directory) in Apache Hudi has different versions of part.
SelectQueryInfo & query_info, /// To find needed parts we need to find out latest part file for every partition.
ContextPtr context, /// Part format is usually parquet, but can differ.
QueryProcessingStage::Enum processed_stage, String generateQueryFromKeys(const std::vector<std::string> & keys, const String & format)
size_t max_block_size,
size_t num_streams)
{
StorageS3::updateS3Configuration(context, base_configuration);
return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
}
std::vector<std::string> StorageHudi::getKeysFromS3()
{
std::vector<std::string> keys;
const auto & client = base_configuration.client;
Aws::S3::Model::ListObjectsV2Request request;
Aws::S3::Model::ListObjectsV2Outcome outcome;
bool is_finished{false};
const auto bucket{base_configuration.uri.bucket};
request.SetBucket(bucket);
request.SetPrefix(table_path);
while (!is_finished)
{
outcome = client->ListObjectsV2(request);
if (!outcome.IsSuccess())
throw Exception(
ErrorCodes::S3_ERROR,
"Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}",
quoteString(bucket),
quoteString(table_path),
backQuote(outcome.GetError().GetExceptionName()),
quoteString(outcome.GetError().GetMessage()));
const auto & result_batch = outcome.GetResult().GetContents();
for (const auto & obj : result_batch)
{
const auto & filename = obj.GetKey().substr(table_path.size()); /// Object name without tablepath prefix.
keys.push_back(filename);
LOG_DEBUG(log, "Found file: {}", filename);
}
request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken());
is_finished = !outcome.GetResult().GetIsTruncated();
}
return keys;
}
String StorageHudi::generateQueryFromKeys(const std::vector<std::string> & keys, const String & format)
{ {
/// For each partition path take only latest file. /// For each partition path take only latest file.
struct FileInfo struct FileInfo
@ -187,6 +92,138 @@ String StorageHudi::generateQueryFromKeys(const std::vector<std::string> & keys,
return "{" + list_of_keys + "}"; return "{" + list_of_keys + "}";
} }
std::vector<std::string> getKeysFromS3(const StorageS3::S3Configuration & base_configuration, const std::string & table_path, Poco::Logger * log)
{
std::vector<std::string> keys;
const auto & client = base_configuration.client;
Aws::S3::Model::ListObjectsV2Request request;
Aws::S3::Model::ListObjectsV2Outcome outcome;
bool is_finished{false};
const auto bucket{base_configuration.uri.bucket};
request.SetBucket(bucket);
request.SetPrefix(table_path);
while (!is_finished)
{
outcome = client->ListObjectsV2(request);
if (!outcome.IsSuccess())
throw Exception(
ErrorCodes::S3_ERROR,
"Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}",
quoteString(bucket),
quoteString(table_path),
backQuote(outcome.GetError().GetExceptionName()),
quoteString(outcome.GetError().GetMessage()));
const auto & result_batch = outcome.GetResult().GetContents();
for (const auto & obj : result_batch)
{
const auto & filename = obj.GetKey().substr(table_path.size()); /// Object name without tablepath prefix.
keys.push_back(filename);
LOG_DEBUG(log, "Found file: {}", filename);
}
request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken());
is_finished = !outcome.GetResult().GetIsTruncated();
}
return keys;
}
StorageS3Configuration getAdjustedS3Configuration(
StorageS3::S3Configuration & base_configuration,
const StorageS3Configuration & configuration,
const std::string & table_path,
Poco::Logger * log)
{
auto keys = getKeysFromS3(base_configuration, table_path, log);
auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(keys, configuration.format);
LOG_DEBUG(log, "New uri: {}", new_uri);
LOG_DEBUG(log, "Table path: {}", table_path);
StorageS3Configuration new_configuration;
new_configuration.url = new_uri;
new_configuration.auth_settings.access_key_id = configuration.auth_settings.access_key_id;
new_configuration.auth_settings.secret_access_key = configuration.auth_settings.secret_access_key;
new_configuration.format = configuration.format;
return new_configuration;
}
}
StorageHudi::StorageHudi(
const StorageS3Configuration & configuration_,
const StorageID & table_id_,
ColumnsDescription columns_,
const ConstraintsDescription & constraints_,
const String & comment,
ContextPtr context_,
std::optional<FormatSettings> format_settings_)
: IStorage(table_id_)
, base_configuration{getBaseConfiguration(configuration_)}
, log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")"))
, table_path(base_configuration.uri.key)
{
StorageInMemoryMetadata storage_metadata;
StorageS3::updateS3Configuration(context_, base_configuration);
auto new_configuration = getAdjustedS3Configuration(base_configuration, configuration_, table_path, log);
if (columns_.empty())
{
columns_ = StorageS3::getTableStructureFromData(
new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr);
storage_metadata.setColumns(columns_);
}
else
storage_metadata.setColumns(columns_);
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata);
s3engine = std::make_shared<StorageS3>(
new_configuration,
table_id_,
columns_,
constraints_,
comment,
context_,
format_settings_,
/* distributed_processing_ */ false,
nullptr);
}
Pipe StorageHudi::read(
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams)
{
StorageS3::updateS3Configuration(context, base_configuration);
return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
}
ColumnsDescription StorageHudi::getTableStructureFromData(
const StorageS3Configuration & configuration, const std::optional<FormatSettings> & format_settings, ContextPtr ctx)
{
auto base_configuration = getBaseConfiguration(configuration);
StorageS3::updateS3Configuration(ctx, base_configuration);
auto new_configuration = getAdjustedS3Configuration(
base_configuration, configuration, base_configuration.uri.key, &Poco::Logger::get("StorageDeltaLake"));
return StorageS3::getTableStructureFromData(
new_configuration, /*distributed processing*/ false, format_settings, ctx, /*object_infos*/ nullptr);
}
void registerStorageHudi(StorageFactory & factory) void registerStorageHudi(StorageFactory & factory)
{ {

View File

@ -48,16 +48,11 @@ public:
size_t max_block_size, size_t max_block_size,
size_t num_streams) override; size_t num_streams) override;
static ColumnsDescription getTableStructureFromData(
const StorageS3Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx);
private: private:
std::vector<std::string> getKeysFromS3();
/// Apache Hudi store parts of data in different files.
/// Every part file has timestamp in it.
/// Every partition(directory) in Apache Hudi has different versions of part.
/// To find needed parts we need to find out latest part file for every partition.
/// Part format is usually parquet, but can differ.
static String generateQueryFromKeys(const std::vector<std::string> & keys, const String & format);
StorageS3::S3Configuration base_configuration; StorageS3::S3Configuration base_configuration;
std::shared_ptr<StorageS3> s3engine; std::shared_ptr<StorageS3> s3engine;
Poco::Logger * log; Poco::Logger * log;

View File

@ -214,7 +214,7 @@ private:
friend class StorageS3Cluster; friend class StorageS3Cluster;
friend class TableFunctionS3Cluster; friend class TableFunctionS3Cluster;
friend class StorageHudi; friend class StorageHudi;
friend class StorageDelta; friend class StorageDeltaLake;
S3Configuration s3_configuration; S3Configuration s3_configuration;
std::vector<String> keys; std::vector<String> keys;

View File

@ -25,6 +25,10 @@ StorageSystemDisks::StorageSystemDisks(const StorageID & table_id_)
{"keep_free_space", std::make_shared<DataTypeUInt64>()}, {"keep_free_space", std::make_shared<DataTypeUInt64>()},
{"type", std::make_shared<DataTypeString>()}, {"type", std::make_shared<DataTypeString>()},
{"is_encrypted", std::make_shared<DataTypeUInt8>()}, {"is_encrypted", std::make_shared<DataTypeUInt8>()},
{"is_read_only", std::make_shared<DataTypeUInt8>()},
{"is_write_once", std::make_shared<DataTypeUInt8>()},
{"is_remote", std::make_shared<DataTypeUInt8>()},
{"is_broken", std::make_shared<DataTypeUInt8>()},
{"cache_path", std::make_shared<DataTypeString>()}, {"cache_path", std::make_shared<DataTypeString>()},
})); }));
setInMemoryMetadata(storage_metadata); setInMemoryMetadata(storage_metadata);
@ -49,6 +53,10 @@ Pipe StorageSystemDisks::read(
MutableColumnPtr col_keep = ColumnUInt64::create(); MutableColumnPtr col_keep = ColumnUInt64::create();
MutableColumnPtr col_type = ColumnString::create(); MutableColumnPtr col_type = ColumnString::create();
MutableColumnPtr col_is_encrypted = ColumnUInt8::create(); MutableColumnPtr col_is_encrypted = ColumnUInt8::create();
MutableColumnPtr col_is_read_only = ColumnUInt8::create();
MutableColumnPtr col_is_write_once = ColumnUInt8::create();
MutableColumnPtr col_is_remote = ColumnUInt8::create();
MutableColumnPtr col_is_broken = ColumnUInt8::create();
MutableColumnPtr col_cache_path = ColumnString::create(); MutableColumnPtr col_cache_path = ColumnString::create();
for (const auto & [disk_name, disk_ptr] : context->getDisksMap()) for (const auto & [disk_name, disk_ptr] : context->getDisksMap())
@ -62,6 +70,10 @@ Pipe StorageSystemDisks::read(
auto data_source_description = disk_ptr->getDataSourceDescription(); auto data_source_description = disk_ptr->getDataSourceDescription();
col_type->insert(toString(data_source_description.type)); col_type->insert(toString(data_source_description.type));
col_is_encrypted->insert(data_source_description.is_encrypted); col_is_encrypted->insert(data_source_description.is_encrypted);
col_is_read_only->insert(disk_ptr->isReadOnly());
col_is_write_once->insert(disk_ptr->isWriteOnce());
col_is_remote->insert(disk_ptr->isRemote());
col_is_broken->insert(disk_ptr->isBroken());
String cache_path; String cache_path;
if (disk_ptr->supportsCache()) if (disk_ptr->supportsCache())
@ -79,6 +91,10 @@ Pipe StorageSystemDisks::read(
res_columns.emplace_back(std::move(col_keep)); res_columns.emplace_back(std::move(col_keep));
res_columns.emplace_back(std::move(col_type)); res_columns.emplace_back(std::move(col_type));
res_columns.emplace_back(std::move(col_is_encrypted)); res_columns.emplace_back(std::move(col_is_encrypted));
res_columns.emplace_back(std::move(col_is_read_only));
res_columns.emplace_back(std::move(col_is_write_once));
res_columns.emplace_back(std::move(col_is_remote));
res_columns.emplace_back(std::move(col_is_broken));
res_columns.emplace_back(std::move(col_cache_path)); res_columns.emplace_back(std::move(col_cache_path));
UInt64 num_rows = res_columns.at(0)->size(); UInt64 num_rows = res_columns.at(0)->size();

View File

@ -34,7 +34,7 @@ void registerStorageS3(StorageFactory & factory);
void registerStorageCOS(StorageFactory & factory); void registerStorageCOS(StorageFactory & factory);
void registerStorageOSS(StorageFactory & factory); void registerStorageOSS(StorageFactory & factory);
void registerStorageHudi(StorageFactory & factory); void registerStorageHudi(StorageFactory & factory);
void registerStorageDelta(StorageFactory & factory); void registerStorageDeltaLake(StorageFactory & factory);
#endif #endif
#if USE_HDFS #if USE_HDFS
@ -123,7 +123,7 @@ void registerStorages()
registerStorageCOS(factory); registerStorageCOS(factory);
registerStorageOSS(factory); registerStorageOSS(factory);
registerStorageHudi(factory); registerStorageHudi(factory);
registerStorageDelta(factory); registerStorageDeltaLake(factory);
#endif #endif
#if USE_HDFS #if USE_HDFS

View File

@ -86,6 +86,16 @@ private:
struct TableFunctionProperties struct TableFunctionProperties
{ {
Documentation documentation; Documentation documentation;
/** It is determined by the possibility of modifying any data or making requests to arbitrary hostnames.
*
* If users can make a request to an arbitrary hostname, they can get the info from the internal network
* or manipulate internal APIs (say - put some data into Memcached, which is available only in the corporate network).
* This is named "SSRF attack".
* Or a user can use an open ClickHouse server to amplify DoS attacks.
*
* In those cases, the table function should not be allowed in readonly mode.
*/
bool allow_readonly = false; bool allow_readonly = false;
}; };

View File

@ -10,7 +10,7 @@
# include <Interpreters/evaluateConstantExpression.h> # include <Interpreters/evaluateConstantExpression.h>
# include <Interpreters/parseColumnsListForTableFunction.h> # include <Interpreters/parseColumnsListForTableFunction.h>
# include <Parsers/ASTLiteral.h> # include <Parsers/ASTLiteral.h>
# include <Storages/StorageDelta.h> # include <Storages/StorageDeltaLake.h>
# include <Storages/StorageURL.h> # include <Storages/StorageURL.h>
# include <Storages/checkAndGetLiteralArgument.h> # include <Storages/checkAndGetLiteralArgument.h>
# include <TableFunctions/TableFunctionDeltaLake.h> # include <TableFunctions/TableFunctionDeltaLake.h>
@ -27,7 +27,7 @@ namespace ErrorCodes
} }
void TableFunctionDelta::parseArgumentsImpl( void TableFunctionDeltaLake::parseArgumentsImpl(
const String & error_message, ASTs & args, ContextPtr context, StorageS3Configuration & base_configuration) const String & error_message, ASTs & args, ContextPtr context, StorageS3Configuration & base_configuration)
{ {
if (args.empty() || args.size() > 6) if (args.empty() || args.size() > 6)
@ -100,7 +100,7 @@ void TableFunctionDelta::parseArgumentsImpl(
= checkAndGetLiteralArgument<String>(args[args_to_idx["secret_access_key"]], "secret_access_key"); = checkAndGetLiteralArgument<String>(args[args_to_idx["secret_access_key"]], "secret_access_key");
} }
void TableFunctionDelta::parseArguments(const ASTPtr & ast_function, ContextPtr context) void TableFunctionDeltaLake::parseArguments(const ASTPtr & ast_function, ContextPtr context)
{ {
/// Parse args /// Parse args
ASTs & args_func = ast_function->children; ASTs & args_func = ast_function->children;
@ -125,18 +125,18 @@ void TableFunctionDelta::parseArguments(const ASTPtr & ast_function, ContextPtr
parseArgumentsImpl(message, args, context, configuration); parseArgumentsImpl(message, args, context, configuration);
} }
ColumnsDescription TableFunctionDelta::getActualTableStructure(ContextPtr context) const ColumnsDescription TableFunctionDeltaLake::getActualTableStructure(ContextPtr context) const
{ {
if (configuration.structure == "auto") if (configuration.structure == "auto")
{ {
context->checkAccess(getSourceAccessType()); context->checkAccess(getSourceAccessType());
return StorageS3::getTableStructureFromData(configuration, false, std::nullopt, context); return StorageDeltaLake::getTableStructureFromData(configuration, std::nullopt, context);
} }
return parseColumnsListFromString(configuration.structure, context); return parseColumnsListFromString(configuration.structure, context);
} }
StoragePtr TableFunctionDelta::executeImpl( StoragePtr TableFunctionDeltaLake::executeImpl(
const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
{ {
Poco::URI uri(configuration.url); Poco::URI uri(configuration.url);
@ -146,7 +146,7 @@ StoragePtr TableFunctionDelta::executeImpl(
if (configuration.structure != "auto") if (configuration.structure != "auto")
columns = parseColumnsListFromString(configuration.structure, context); columns = parseColumnsListFromString(configuration.structure, context);
StoragePtr storage = std::make_shared<StorageDelta>( StoragePtr storage = std::make_shared<StorageDeltaLake>(
configuration, StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription{}, String{}, context, std::nullopt); configuration, StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription{}, String{}, context, std::nullopt);
storage->startup(); storage->startup();
@ -155,9 +155,9 @@ StoragePtr TableFunctionDelta::executeImpl(
} }
void registerTableFunctionDelta(TableFunctionFactory & factory) void registerTableFunctionDeltaLake(TableFunctionFactory & factory)
{ {
factory.registerFunction<TableFunctionDelta>( factory.registerFunction<TableFunctionDeltaLake>(
{.documentation {.documentation
= {R"(The table function can be used to read the DeltaLake table stored on object store.)", = {R"(The table function can be used to read the DeltaLake table stored on object store.)",
Documentation::Examples{{"deltaLake", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)"}}, Documentation::Examples{{"deltaLake", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)"}},

View File

@ -16,7 +16,7 @@ class TableFunctionS3Cluster;
/* deltaLake(source, [access_key_id, secret_access_key,] format, structure[, compression]) - creates a temporary DeltaLake table on S3. /* deltaLake(source, [access_key_id, secret_access_key,] format, structure[, compression]) - creates a temporary DeltaLake table on S3.
*/ */
class TableFunctionDelta : public ITableFunction class TableFunctionDeltaLake : public ITableFunction
{ {
public: public:
static constexpr auto name = "deltaLake"; static constexpr auto name = "deltaLake";

View File

@ -89,9 +89,72 @@ StoragePtr TableFunctionFormat::executeImpl(const ASTPtr & /*ast_function*/, Con
return res; return res;
} }
static const Documentation format_table_function_documentation =
{
R"(
Extracts table structure from data and parses it according to specified input format.
Syntax: `format(format_name, data)`.
Parameters:
- `format_name` - the format of the data.
- `data ` - String literal or constant expression that returns a string containing data in specified format.
Returned value: A table with data parsed from `data` argument according specified format and extracted schema.
)",
Documentation::Examples
{
{
"First example",
R"(
Query:
```
:) select * from format(JSONEachRow,
$$
{"a": "Hello", "b": 111}
{"a": "World", "b": 123}
{"a": "Hello", "b": 112}
{"a": "World", "b": 124}
$$)
```
Result:
```
ba
111 Hello
123 World
112 Hello
124 World
```
)"
},
{
"Second example",
R"(
Query:
```
:) desc format(JSONEachRow,
$$
{"a": "Hello", "b": 111}
{"a": "World", "b": 123}
{"a": "Hello", "b": 112}
{"a": "World", "b": 124}
$$)
```
Result:
```
nametypedefault_typedefault_expressioncommentcodec_expressionttl_expression
b Nullable(Float64)
a Nullable(String)
```
)"
},
},
Documentation::Categories{"format", "table-functions"}
};
void registerTableFunctionFormat(TableFunctionFactory & factory) void registerTableFunctionFormat(TableFunctionFactory & factory)
{ {
factory.registerFunction<TableFunctionFormat>({}, TableFunctionFactory::CaseInsensitive); factory.registerFunction<TableFunctionFormat>({format_table_function_documentation, false}, TableFunctionFactory::CaseInsensitive);
} }
} }

View File

@ -130,7 +130,7 @@ ColumnsDescription TableFunctionHudi::getActualTableStructure(ContextPtr context
if (configuration.structure == "auto") if (configuration.structure == "auto")
{ {
context->checkAccess(getSourceAccessType()); context->checkAccess(getSourceAccessType());
return StorageS3::getTableStructureFromData(configuration, false, std::nullopt, context); return StorageHudi::getTableStructureFromData(configuration, std::nullopt, context);
} }
return parseColumnsListFromString(configuration.structure, context); return parseColumnsListFromString(configuration.structure, context);

View File

@ -28,7 +28,7 @@ void registerTableFunctions()
registerTableFunctionS3Cluster(factory); registerTableFunctionS3Cluster(factory);
registerTableFunctionCOS(factory); registerTableFunctionCOS(factory);
registerTableFunctionHudi(factory); registerTableFunctionHudi(factory);
registerTableFunctionDelta(factory); registerTableFunctionDeltaLake(factory);
registerTableFunctionOSS(factory); registerTableFunctionOSS(factory);
#endif #endif

View File

@ -25,7 +25,7 @@ void registerTableFunctionS3(TableFunctionFactory & factory);
void registerTableFunctionS3Cluster(TableFunctionFactory & factory); void registerTableFunctionS3Cluster(TableFunctionFactory & factory);
void registerTableFunctionCOS(TableFunctionFactory & factory); void registerTableFunctionCOS(TableFunctionFactory & factory);
void registerTableFunctionHudi(TableFunctionFactory & factory); void registerTableFunctionHudi(TableFunctionFactory & factory);
void registerTableFunctionDelta(TableFunctionFactory & factory); void registerTableFunctionDeltaLake(TableFunctionFactory & factory);
void registerTableFunctionOSS(TableFunctionFactory & factory); void registerTableFunctionOSS(TableFunctionFactory & factory);
#endif #endif

View File

@ -1,13 +0,0 @@
FROM public.ecr.aws/lambda/python:3.9
# Install the function's dependencies using file requirements.txt
# from your project folder.
COPY requirements.txt .
RUN pip3 install -r requirements.txt --target "${LAMBDA_TASK_ROOT}"
# Copy function code
COPY app.py ${LAMBDA_TASK_ROOT}
# Set the CMD to your handler (could also be done as a parameter override outside of the Dockerfile)
CMD [ "app.handler" ]

View File

@ -1,5 +1,6 @@
#!/usr/bin/env python3 #!/usr/bin/env python3
from base64 import b64decode
from collections import namedtuple from collections import namedtuple
from typing import Any, Dict, List from typing import Any, Dict, List
from threading import Thread from threading import Thread
@ -19,26 +20,25 @@ NEED_RERUN_OR_CANCELL_WORKFLOWS = {
"BackportPR", "BackportPR",
} }
# https://docs.github.com/en/rest/reference/actions#cancel-a-workflow-run
#
API_URL = os.getenv("API_URL", "https://api.github.com/repos/ClickHouse/ClickHouse")
MAX_RETRY = 5 MAX_RETRY = 5
DEBUG_INFO = {} # type: Dict[str, Any] DEBUG_INFO = {} # type: Dict[str, Any]
class Worker(Thread): class Worker(Thread):
def __init__(self, request_queue: Queue, ignore_exception: bool = False): def __init__(
self, request_queue: Queue, token: str, ignore_exception: bool = False
):
Thread.__init__(self) Thread.__init__(self)
self.queue = request_queue self.queue = request_queue
self.token = token
self.ignore_exception = ignore_exception self.ignore_exception = ignore_exception
self.response = {} # type: Dict self.response = {} # type: Dict
def run(self): def run(self):
m = self.queue.get() m = self.queue.get()
try: try:
self.response = _exec_get_with_retry(m) self.response = _exec_get_with_retry(m, self.token)
except Exception as e: except Exception as e:
if not self.ignore_exception: if not self.ignore_exception:
raise raise
@ -98,10 +98,11 @@ def get_token_from_aws():
return get_access_token(encoded_jwt, installation_id) return get_access_token(encoded_jwt, installation_id)
def _exec_get_with_retry(url): def _exec_get_with_retry(url: str, token: str) -> dict:
headers = {"Authorization": f"token {token}"}
for i in range(MAX_RETRY): for i in range(MAX_RETRY):
try: try:
response = requests.get(url) response = requests.get(url, headers=headers)
response.raise_for_status() response.raise_for_status()
return response.json() return response.json()
except Exception as ex: except Exception as ex:
@ -113,23 +114,25 @@ def _exec_get_with_retry(url):
WorkflowDescription = namedtuple( WorkflowDescription = namedtuple(
"WorkflowDescription", "WorkflowDescription",
["run_id", "head_sha", "status", "rerun_url", "cancel_url", "conclusion"], ["url", "run_id", "head_sha", "status", "rerun_url", "cancel_url", "conclusion"],
) )
def get_workflows_description_for_pull_request( def get_workflows_description_for_pull_request(
pull_request_event, pull_request_event,
token,
) -> List[WorkflowDescription]: ) -> List[WorkflowDescription]:
head_repo = pull_request_event["head"]["repo"]["full_name"] head_repo = pull_request_event["head"]["repo"]["full_name"]
head_branch = pull_request_event["head"]["ref"] head_branch = pull_request_event["head"]["ref"]
print("PR", pull_request_event["number"], "has head ref", head_branch) print("PR", pull_request_event["number"], "has head ref", head_branch)
workflows_data = [] workflows_data = []
request_url = f"{API_URL}/actions/runs?per_page=100" repo_url = pull_request_event["base"]["repo"]["url"]
request_url = f"{repo_url}/actions/runs?per_page=100"
# Get all workflows for the current branch # Get all workflows for the current branch
for i in range(1, 11): for i in range(1, 11):
workflows = _exec_get_with_retry( workflows = _exec_get_with_retry(
f"{request_url}&event=pull_request&branch={head_branch}&page={i}" f"{request_url}&event=pull_request&branch={head_branch}&page={i}", token
) )
if not workflows["workflow_runs"]: if not workflows["workflow_runs"]:
break break
@ -164,6 +167,7 @@ def get_workflows_description_for_pull_request(
): ):
workflow_descriptions.append( workflow_descriptions.append(
WorkflowDescription( WorkflowDescription(
url=workflow["url"],
run_id=workflow["id"], run_id=workflow["id"],
head_sha=workflow["head_sha"], head_sha=workflow["head_sha"],
status=workflow["status"], status=workflow["status"],
@ -176,19 +180,22 @@ def get_workflows_description_for_pull_request(
return workflow_descriptions return workflow_descriptions
def get_workflow_description_fallback(pull_request_event) -> List[WorkflowDescription]: def get_workflow_description_fallback(
pull_request_event, token
) -> List[WorkflowDescription]:
head_repo = pull_request_event["head"]["repo"]["full_name"] head_repo = pull_request_event["head"]["repo"]["full_name"]
head_branch = pull_request_event["head"]["ref"] head_branch = pull_request_event["head"]["ref"]
print("Get last 500 workflows from API to search related there") print("Get last 500 workflows from API to search related there")
# Fallback for a case of an already deleted branch and no workflows received # Fallback for a case of an already deleted branch and no workflows received
request_url = f"{API_URL}/actions/runs?per_page=100" repo_url = pull_request_event["base"]["repo"]["url"]
request_url = f"{repo_url}/actions/runs?per_page=100"
q = Queue() # type: Queue q = Queue() # type: Queue
workers = [] workers = []
workflows_data = [] workflows_data = []
i = 1 i = 1
for i in range(1, 6): for i in range(1, 6):
q.put(f"{request_url}&page={i}") q.put(f"{request_url}&page={i}")
worker = Worker(q, True) worker = Worker(q, token, True)
worker.start() worker.start()
workers.append(worker) workers.append(worker)
@ -220,6 +227,7 @@ def get_workflow_description_fallback(pull_request_event) -> List[WorkflowDescri
workflow_descriptions = [ workflow_descriptions = [
WorkflowDescription( WorkflowDescription(
url=wf["url"],
run_id=wf["id"], run_id=wf["id"],
head_sha=wf["head_sha"], head_sha=wf["head_sha"],
status=wf["status"], status=wf["status"],
@ -233,9 +241,10 @@ def get_workflow_description_fallback(pull_request_event) -> List[WorkflowDescri
return workflow_descriptions return workflow_descriptions
def get_workflow_description(workflow_id) -> WorkflowDescription: def get_workflow_description(workflow_url, token) -> WorkflowDescription:
workflow = _exec_get_with_retry(API_URL + f"/actions/runs/{workflow_id}") workflow = _exec_get_with_retry(workflow_url, token)
return WorkflowDescription( return WorkflowDescription(
url=workflow["url"],
run_id=workflow["id"], run_id=workflow["id"],
head_sha=workflow["head_sha"], head_sha=workflow["head_sha"],
status=workflow["status"], status=workflow["status"],
@ -268,8 +277,11 @@ def exec_workflow_url(urls_to_cancel, token):
def main(event): def main(event):
token = get_token_from_aws() token = get_token_from_aws()
DEBUG_INFO["event_body"] = event["body"] DEBUG_INFO["event"] = event
event_data = json.loads(event["body"]) if event["isBase64Encoded"]:
event_data = json.loads(b64decode(event["body"]))
else:
event_data = json.loads(event["body"])
print("Got event for PR", event_data["number"]) print("Got event for PR", event_data["number"])
action = event_data["action"] action = event_data["action"]
@ -279,9 +291,12 @@ def main(event):
print("PR has labels", labels) print("PR has labels", labels)
if action == "closed" or "do not test" in labels: if action == "closed" or "do not test" in labels:
print("PR merged/closed or manually labeled 'do not test' will kill workflows") print("PR merged/closed or manually labeled 'do not test' will kill workflows")
workflow_descriptions = get_workflows_description_for_pull_request(pull_request) workflow_descriptions = get_workflows_description_for_pull_request(
pull_request, token
)
workflow_descriptions = ( workflow_descriptions = (
workflow_descriptions or get_workflow_description_fallback(pull_request) workflow_descriptions
or get_workflow_description_fallback(pull_request, token)
) )
urls_to_cancel = [] urls_to_cancel = []
for workflow_description in workflow_descriptions: for workflow_description in workflow_descriptions:
@ -294,9 +309,12 @@ def main(event):
exec_workflow_url(urls_to_cancel, token) exec_workflow_url(urls_to_cancel, token)
elif action == "synchronize": elif action == "synchronize":
print("PR is synchronized, going to stop old actions") print("PR is synchronized, going to stop old actions")
workflow_descriptions = get_workflows_description_for_pull_request(pull_request) workflow_descriptions = get_workflows_description_for_pull_request(
pull_request, token
)
workflow_descriptions = ( workflow_descriptions = (
workflow_descriptions or get_workflow_description_fallback(pull_request) workflow_descriptions
or get_workflow_description_fallback(pull_request, token)
) )
urls_to_cancel = [] urls_to_cancel = []
for workflow_description in workflow_descriptions: for workflow_description in workflow_descriptions:
@ -308,11 +326,14 @@ def main(event):
urls_to_cancel.append(workflow_description.cancel_url) urls_to_cancel.append(workflow_description.cancel_url)
print(f"Found {len(urls_to_cancel)} workflows to cancel") print(f"Found {len(urls_to_cancel)} workflows to cancel")
exec_workflow_url(urls_to_cancel, token) exec_workflow_url(urls_to_cancel, token)
elif action == "labeled" and "can be tested" in labels: elif action == "labeled" and event_data["label"]["name"] == "can be tested":
print("PR marked with can be tested label, rerun workflow") print("PR marked with can be tested label, rerun workflow")
workflow_descriptions = get_workflows_description_for_pull_request(pull_request) workflow_descriptions = get_workflows_description_for_pull_request(
pull_request, token
)
workflow_descriptions = ( workflow_descriptions = (
workflow_descriptions or get_workflow_description_fallback(pull_request) workflow_descriptions
or get_workflow_description_fallback(pull_request, token)
) )
if not workflow_descriptions: if not workflow_descriptions:
print("Not found any workflows") print("Not found any workflows")
@ -330,7 +351,10 @@ def main(event):
print("Cancelled") print("Cancelled")
for _ in range(45): for _ in range(45):
latest_workflow_desc = get_workflow_description(most_recent_workflow.run_id) # If the number of retries is changed: tune the lambda limits accordingly
latest_workflow_desc = get_workflow_description(
most_recent_workflow.url, token
)
print("Checking latest workflow", latest_workflow_desc) print("Checking latest workflow", latest_workflow_desc)
if latest_workflow_desc.status in ("completed", "cancelled"): if latest_workflow_desc.status in ("completed", "cancelled"):
print("Finally latest workflow done, going to rerun") print("Finally latest workflow done, going to rerun")
@ -347,6 +371,12 @@ def main(event):
def handler(event, _): def handler(event, _):
try: try:
main(event) main(event)
return {
"statusCode": 200,
"headers": {"Content-Type": "application/json"},
"body": '{"status": "OK"}',
}
finally: finally:
for name, value in DEBUG_INFO.items(): for name, value in DEBUG_INFO.items():
print(f"Value of {name}: ", value) print(f"Value of {name}: ", value)

View File

@ -0,0 +1 @@
../team_keys_lambda/build_and_deploy_archive.sh

View File

@ -1,4 +1,6 @@
#!/usr/bin/env python3 #!/usr/bin/env python3
import logging
import boto3 # type: ignore import boto3 # type: ignore
from github import Github # type: ignore from github import Github # type: ignore
@ -9,14 +11,30 @@ def get_parameter_from_ssm(name, decrypt=True, client=None):
return client.get_parameter(Name=name, WithDecryption=decrypt)["Parameter"]["Value"] return client.get_parameter(Name=name, WithDecryption=decrypt)["Parameter"]["Value"]
def get_best_robot_token(token_prefix_env_name="github_robot_token_", total_tokens=4): def get_best_robot_token(token_prefix_env_name="github_robot_token_"):
client = boto3.client("ssm", region_name="us-east-1") client = boto3.client("ssm", region_name="us-east-1")
tokens = {} parameters = client.describe_parameters(
for i in range(1, total_tokens + 1): ParameterFilters=[
token_name = token_prefix_env_name + str(i) {"Key": "Name", "Option": "BeginsWith", "Values": [token_prefix_env_name]}
token = get_parameter_from_ssm(token_name, True, client) ]
gh = Github(token, per_page=100) )["Parameters"]
rest, _ = gh.rate_limiting assert parameters
tokens[token] = rest token = {"login": "", "value": "", "rest": 0}
return max(tokens.items(), key=lambda x: x[1])[0] for token_name in [p["Name"] for p in parameters]:
value = get_parameter_from_ssm(token_name, True, client)
gh = Github(value, per_page=100)
# Do not spend additional request to API by accessin user.login unless
# the token is chosen by the remaining requests number
user = gh.get_user()
rest, _ = gh.rate_limiting
logging.info("Get token with %s remaining requests", rest)
if token["rest"] < rest:
token = {"user": user, "value": value, "rest": rest}
assert token["value"]
logging.info(
"User %s with %s remaining requests is used", token["user"].login, token["rest"]
)
return token["value"]

View File

@ -491,6 +491,12 @@ def main(event):
def handler(event, _): def handler(event, _):
try: try:
main(event) main(event)
return {
"statusCode": 200,
"headers": {"Content-Type": "application/json"},
"body": '{"status": "OK"}',
}
except Exception: except Exception:
print("Received event: ", event) print("Received event: ", event)
raise raise

View File

@ -3332,7 +3332,7 @@ class ClickHouseInstance:
except Exception as e: except Exception as e:
logging.warning(f"Stop ClickHouse raised an error {e}") logging.warning(f"Stop ClickHouse raised an error {e}")
def start_clickhouse(self, start_wait_sec=60): def start_clickhouse(self, start_wait_sec=60, retry_start=True):
if not self.stay_alive: if not self.stay_alive:
raise Exception( raise Exception(
"ClickHouse can be started again only with stay_alive=True instance" "ClickHouse can be started again only with stay_alive=True instance"
@ -3364,6 +3364,8 @@ class ClickHouseInstance:
self.exec_in_container( self.exec_in_container(
["bash", "-c", f"kill -9 {pid}"], user="root", nothrow=True ["bash", "-c", f"kill -9 {pid}"], user="root", nothrow=True
) )
if not retry_start:
raise
time.sleep(time_to_sleep) time.sleep(time_to_sleep)
raise Exception("Cannot start ClickHouse, see additional info in logs") raise Exception("Cannot start ClickHouse, see additional info in logs")

View File

@ -39,3 +39,15 @@ def wait_until_quorum_lost(cluster, node, port=9181):
def wait_nodes(cluster, nodes): def wait_nodes(cluster, nodes):
for node in nodes: for node in nodes:
wait_until_connected(cluster, node) wait_until_connected(cluster, node)
def is_leader(cluster, node, port=9181):
stat = send_4lw_cmd(cluster, node, "stat", port)
return "Mode: leader" in stat
def get_leader(cluster, nodes):
for node in nodes:
if is_leader(cluster, node):
return node
raise Exception("No leader in Keeper cluster.")

View File

@ -30,9 +30,7 @@ def start_cluster():
pytest.param("wide", "backup_wide", "s3_backup_wide", int(0), id="wide"), pytest.param("wide", "backup_wide", "s3_backup_wide", int(0), id="wide"),
], ],
) )
def test_attach_compact_part( def test_attach_part(table_name, backup_name, storage_policy, min_bytes_for_wide_part):
table_name, backup_name, storage_policy, min_bytes_for_wide_part
):
node.query( node.query(
f""" f"""
-- Catch any errors (NOTE: warnings are ok) -- Catch any errors (NOTE: warnings are ok)
@ -61,9 +59,6 @@ def test_attach_compact_part(
node.query( node.query(
f""" f"""
-- NOTE: be aware not to DROP the table, but DETACH first to keep it in S3.
detach table ordinary_db.{table_name};
-- NOTE: DROP DATABASE cannot be done w/o this due to metadata leftovers -- NOTE: DROP DATABASE cannot be done w/o this due to metadata leftovers
set force_remove_data_recursively_on_drop=1; set force_remove_data_recursively_on_drop=1;
drop database ordinary_db sync; drop database ordinary_db sync;

View File

@ -13,7 +13,11 @@ upstream = cluster.add_instance("upstream")
backward = cluster.add_instance( backward = cluster.add_instance(
"backward", "backward",
image="clickhouse/clickhouse-server", image="clickhouse/clickhouse-server",
tag="22.9", # Note that a bug changed the string representation of several aggregations in 22.9 and 22.10 and some minor
# releases of 22.8, 22.7 and 22.3
# See https://github.com/ClickHouse/ClickHouse/issues/42916
# Affected at least: singleValueOrNull, last_value, min, max, any, anyLast, anyHeavy, first_value, argMin, argMax
tag="22.6",
with_installed_binary=True, with_installed_binary=True,
) )
@ -139,6 +143,9 @@ def test_string_functions(start_cluster):
"substring", "substring",
"CAST", "CAST",
# NOTE: no need to ignore now()/now64() since they will fail because they don't accept any argument # NOTE: no need to ignore now()/now64() since they will fail because they don't accept any argument
# 22.8 Backward Incompatible Change: Extended range of Date32
"toDate32OrZero",
"toDate32OrDefault",
] ]
functions = filter(lambda x: x not in excludes, functions) functions = filter(lambda x: x not in excludes, functions)
@ -149,14 +156,15 @@ def test_string_functions(start_cluster):
failed = 0 failed = 0
passed = 0 passed = 0
def get_function_value(node, function_name, value="foo"): def get_function_value(node, function_name, value):
return node.query(f"select {function_name}('{value}')").strip() return node.query(f"select {function_name}('{value}')").strip()
v = "foo"
for function in functions: for function in functions:
logging.info("Checking %s", function) logging.info("Checking %s('%s')", function, v)
try: try:
backward_value = get_function_value(backward, function) backward_value = get_function_value(backward, function, v)
except QueryRuntimeException as e: except QueryRuntimeException as e:
error_message = str(e) error_message = str(e)
allowed_errors = [ allowed_errors = [
@ -199,11 +207,12 @@ def test_string_functions(start_cluster):
failed += 1 failed += 1
continue continue
upstream_value = get_function_value(upstream, function) upstream_value = get_function_value(upstream, function, v)
if upstream_value != backward_value: if upstream_value != backward_value:
logging.info( logging.warning(
"Failed %s, %s (backward) != %s (upstream)", "Failed %s('%s') %s (backward) != %s (upstream)",
function, function,
v,
backward_value, backward_value,
upstream_value, upstream_value,
) )

View File

@ -13,7 +13,10 @@ def cluster():
"node1", main_configs=["configs/storage_conf.xml"], with_nginx=True "node1", main_configs=["configs/storage_conf.xml"], with_nginx=True
) )
cluster.add_instance( cluster.add_instance(
"node2", main_configs=["configs/storage_conf_web.xml"], with_nginx=True "node2",
main_configs=["configs/storage_conf_web.xml"],
with_nginx=True,
stay_alive=True,
) )
cluster.add_instance( cluster.add_instance(
"node3", main_configs=["configs/storage_conf_web.xml"], with_nginx=True "node3", main_configs=["configs/storage_conf_web.xml"], with_nginx=True
@ -192,3 +195,53 @@ def test_cache(cluster, node_name):
node2.query("DROP TABLE test{} SYNC".format(i)) node2.query("DROP TABLE test{} SYNC".format(i))
print(f"Ok {i}") print(f"Ok {i}")
def test_unavailable_server(cluster):
"""
Regression test for the case when clickhouse-server simply ignore when
server is unavailable on start and later will simply return 0 rows for
SELECT from table on web disk.
"""
node2 = cluster.instances["node2"]
global uuids
node2.query(
"""
ATTACH TABLE test0 UUID '{}'
(id Int32) ENGINE = MergeTree() ORDER BY id
SETTINGS storage_policy = 'web';
""".format(
uuids[0]
)
)
node2.stop_clickhouse()
try:
# NOTE: you cannot use separate disk instead, since MergeTree engine will
# try to lookup parts on all disks (to look unexpected disks with parts)
# and fail because of unavailable server.
node2.exec_in_container(
[
"bash",
"-c",
"sed -i 's#http://nginx:80/test1/#http://nginx:8080/test1/#' /etc/clickhouse-server/config.d/storage_conf_web.xml",
]
)
with pytest.raises(Exception):
# HTTP retries with backup can take awhile
node2.start_clickhouse(start_wait_sec=120, retry_start=False)
assert node2.contains_in_log(
"Caught exception while loading metadata.*Connection refused"
)
assert node2.contains_in_log(
"HTTP request to \`http://nginx:8080/test1/.*\` failed at try 1/10 with bytes read: 0/unknown. Error: Connection refused."
)
finally:
node2.exec_in_container(
[
"bash",
"-c",
"sed -i 's#http://nginx:8080/test1/#http://nginx:80/test1/#' /etc/clickhouse-server/config.d/storage_conf_web.xml",
]
)
node2.start_clickhouse()
node2.query("DROP TABLE test0 SYNC")

View File

@ -12,7 +12,7 @@
</disk_memory> </disk_memory>
<disk_hdfs> <disk_hdfs>
<type>hdfs</type> <type>hdfs</type>
<endpoint>hdfs://hdfs1:9000/data/</endpoint> <endpoint>hdfs://hdfs1:9000/</endpoint>
</disk_hdfs> </disk_hdfs>
<disk_encrypted> <disk_encrypted>
<type>encrypted</type> <type>encrypted</type>

View File

@ -22,6 +22,7 @@ def cluster():
with_hdfs=True, with_hdfs=True,
) )
cluster.start() cluster.start()
yield cluster yield cluster
finally: finally:
cluster.shutdown() cluster.shutdown()

View File

@ -1,3 +1,4 @@
<clickhouse> <clickhouse>
<max_server_memory_usage>2000000000</max_server_memory_usage> <max_server_memory_usage>2000000000</max_server_memory_usage>
<allow_use_jemalloc_memory>false</allow_use_jemalloc_memory>
</clickhouse> </clickhouse>

View File

@ -33,7 +33,7 @@
<id>3</id> <id>3</id>
<hostname>node3</hostname> <hostname>node3</hostname>
<port>9234</port> <port>9234</port>
<can_become_leader>false</can_become_leader> <can_become_leader>true</can_become_leader>
<start_as_follower>true</start_as_follower> <start_as_follower>true</start_as_follower>
<priority>1</priority> <priority>1</priority>
</server> </server>

View File

@ -33,7 +33,7 @@
<id>3</id> <id>3</id>
<hostname>node3</hostname> <hostname>node3</hostname>
<port>9234</port> <port>9234</port>
<can_become_leader>false</can_become_leader> <can_become_leader>true</can_become_leader>
<start_as_follower>true</start_as_follower> <start_as_follower>true</start_as_follower>
<priority>1</priority> <priority>1</priority>
</server> </server>

View File

@ -33,7 +33,7 @@
<id>3</id> <id>3</id>
<hostname>node3</hostname> <hostname>node3</hostname>
<port>9234</port> <port>9234</port>
<can_become_leader>false</can_become_leader> <can_become_leader>true</can_become_leader>
<start_as_follower>true</start_as_follower> <start_as_follower>true</start_as_follower>
<priority>1</priority> <priority>1</priority>
</server> </server>

View File

@ -148,10 +148,11 @@ def test_cmd_mntr(started_cluster):
wait_nodes() wait_nodes()
clear_znodes() clear_znodes()
leader = keeper_utils.get_leader(cluster, [node1, node2, node3])
# reset stat first # reset stat first
reset_node_stats(node1) reset_node_stats(leader)
zk = get_fake_zk(node1.name, timeout=30.0) zk = get_fake_zk(leader.name, timeout=30.0)
do_some_action( do_some_action(
zk, zk,
create_cnt=10, create_cnt=10,
@ -162,7 +163,7 @@ def test_cmd_mntr(started_cluster):
delete_cnt=2, delete_cnt=2,
) )
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="mntr") data = keeper_utils.send_4lw_cmd(cluster, leader, cmd="mntr")
# print(data.decode()) # print(data.decode())
reader = csv.reader(data.split("\n"), delimiter="\t") reader = csv.reader(data.split("\n"), delimiter="\t")
@ -307,12 +308,13 @@ def test_cmd_srvr(started_cluster):
wait_nodes() wait_nodes()
clear_znodes() clear_znodes()
reset_node_stats(node1) leader = keeper_utils.get_leader(cluster, [node1, node2, node3])
reset_node_stats(leader)
zk = get_fake_zk(node1.name, timeout=30.0) zk = get_fake_zk(leader.name, timeout=30.0)
do_some_action(zk, create_cnt=10) do_some_action(zk, create_cnt=10)
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="srvr") data = keeper_utils.send_4lw_cmd(cluster, leader, cmd="srvr")
print("srvr output -------------------------------------") print("srvr output -------------------------------------")
print(data) print(data)
@ -329,7 +331,7 @@ def test_cmd_srvr(started_cluster):
assert result["Received"] == "10" assert result["Received"] == "10"
assert result["Sent"] == "10" assert result["Sent"] == "10"
assert int(result["Connections"]) == 1 assert int(result["Connections"]) == 1
assert int(result["Zxid"]) > 14 assert int(result["Zxid"]) > 10
assert result["Mode"] == "leader" assert result["Mode"] == "leader"
assert result["Node count"] == "13" assert result["Node count"] == "13"
@ -342,13 +344,15 @@ def test_cmd_stat(started_cluster):
try: try:
wait_nodes() wait_nodes()
clear_znodes() clear_znodes()
reset_node_stats(node1)
reset_conn_stats(node1)
zk = get_fake_zk(node1.name, timeout=30.0) leader = keeper_utils.get_leader(cluster, [node1, node2, node3])
reset_node_stats(leader)
reset_conn_stats(leader)
zk = get_fake_zk(leader.name, timeout=30.0)
do_some_action(zk, create_cnt=10) do_some_action(zk, create_cnt=10)
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="stat") data = keeper_utils.send_4lw_cmd(cluster, leader, cmd="stat")
print("stat output -------------------------------------") print("stat output -------------------------------------")
print(data) print(data)
@ -604,6 +608,10 @@ def test_cmd_csnp(started_cluster):
wait_nodes() wait_nodes()
zk = get_fake_zk(node1.name, timeout=30.0) zk = get_fake_zk(node1.name, timeout=30.0)
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="csnp") data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="csnp")
print("csnp output -------------------------------------")
print(data)
try: try:
int(data) int(data)
assert True assert True
@ -623,7 +631,10 @@ def test_cmd_lgif(started_cluster):
do_some_action(zk, create_cnt=100) do_some_action(zk, create_cnt=100)
data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="lgif") data = keeper_utils.send_4lw_cmd(cluster, node1, cmd="lgif")
print("lgif output -------------------------------------")
print(data) print(data)
reader = csv.reader(data.split("\n"), delimiter="\t") reader = csv.reader(data.split("\n"), delimiter="\t")
result = {} result = {}
@ -641,3 +652,28 @@ def test_cmd_lgif(started_cluster):
assert int(result["last_snapshot_idx"]) >= 1 assert int(result["last_snapshot_idx"]) >= 1
finally: finally:
destroy_zk_client(zk) destroy_zk_client(zk)
def test_cmd_rqld(started_cluster):
wait_nodes()
# node2 can not be leader
for node in [node1, node3]:
data = keeper_utils.send_4lw_cmd(cluster, node, cmd="rqld")
assert data == "Sent leadership request to leader."
print("rqld output -------------------------------------")
print(data)
if not keeper_utils.is_leader(cluster, node):
# pull wait to become leader
retry = 0
# TODO not a restrict way
while not keeper_utils.is_leader(cluster, node) and retry < 30:
time.sleep(1)
retry += 1
if retry == 30:
print(
node.name
+ " does not become leader after 30s, maybe there is something wrong."
)
assert keeper_utils.is_leader(cluster, node)

View File

@ -4,6 +4,8 @@
<hdfs> <hdfs>
<type>hdfs</type> <type>hdfs</type>
<endpoint>hdfs://hdfs1:9000/clickhouse/</endpoint> <endpoint>hdfs://hdfs1:9000/clickhouse/</endpoint>
<!-- FIXME: chicken and egg problem with current cluster.py -->
<skip_access_check>true</skip_access_check>
</hdfs> </hdfs>
</disks> </disks>
</storage_configuration> </storage_configuration>

View File

@ -4,6 +4,8 @@
<hdfs> <hdfs>
<type>hdfs</type> <type>hdfs</type>
<endpoint>hdfs://hdfs1:9000/clickhouse/</endpoint> <endpoint>hdfs://hdfs1:9000/clickhouse/</endpoint>
<!-- FIXME: chicken and egg problem with current cluster.py -->
<skip_access_check>true</skip_access_check>
</hdfs> </hdfs>
<hdd> <hdd>
<type>local</type> <type>local</type>

View File

@ -84,6 +84,8 @@ def test_restart_zookeeper(start_cluster):
time.sleep(5) time.sleep(5)
for table_id in range(NUM_TABLES): for table_id in range(NUM_TABLES):
node1.query( node1.query_with_retry(
f"INSERT INTO test_table_{table_id} VALUES (6), (7), (8), (9), (10);" sql=f"INSERT INTO test_table_{table_id} VALUES (6), (7), (8), (9), (10);",
retry_count=10,
sleep_time=1,
) )

View File

@ -4,14 +4,20 @@
<hdfs1> <hdfs1>
<type>hdfs</type> <type>hdfs</type>
<endpoint>hdfs://hdfs1:9000/clickhouse1/</endpoint> <endpoint>hdfs://hdfs1:9000/clickhouse1/</endpoint>
<!-- FIXME: chicken and egg problem with current cluster.py -->
<skip_access_check>true</skip_access_check>
</hdfs1> </hdfs1>
<hdfs1_again> <hdfs1_again>
<type>hdfs</type> <type>hdfs</type>
<endpoint>hdfs://hdfs1:9000/clickhouse1/</endpoint> <endpoint>hdfs://hdfs1:9000/clickhouse1/</endpoint>
<!-- FIXME: chicken and egg problem with current cluster.py -->
<skip_access_check>true</skip_access_check>
</hdfs1_again> </hdfs1_again>
<hdfs2> <hdfs2>
<type>hdfs</type> <type>hdfs</type>
<endpoint>hdfs://hdfs1:9000/clickhouse2/</endpoint> <endpoint>hdfs://hdfs1:9000/clickhouse2/</endpoint>
<!-- FIXME: chicken and egg problem with current cluster.py -->
<skip_access_check>true</skip_access_check>
</hdfs2> </hdfs2>
</disks> </disks>
<policies> <policies>

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