Merge branch 'master' into deprecate-in-memory-parts

This commit is contained in:
Alexey Milovidov 2023-05-05 06:43:23 +02:00
commit 09f171822e
192 changed files with 1900 additions and 1489 deletions

View File

@ -2,7 +2,7 @@
A technical comment, you are free to remove or leave it as it is when PR is created
The following categories are used in the next scripts, update them accordingly
utils/changelog/changelog.py
tests/ci/run_check.py
tests/ci/cancel_and_rerun_workflow_lambda/app.py
-->
### Changelog category (leave one):
- New Feature

View File

@ -134,7 +134,9 @@ add_contrib (aws-cmake
)
add_contrib (base64-cmake base64)
if (NOT ARCH_S390X)
add_contrib (simdjson-cmake simdjson)
endif()
add_contrib (rapidjson-cmake rapidjson)
add_contrib (fastops-cmake fastops)
add_contrib (libuv-cmake libuv)

View File

@ -18,6 +18,9 @@ ENV TIME_LIMIT="30"
ENV KEEPER_NODE=""
ENV NEMESIS=""
ENV WORKLOAD=""
ENV WITH_LOCAL_BINARY=""
ENV RATE=""
ENV CONCURRENCY=""
# volumes

View File

@ -15,12 +15,12 @@ if [ -z "$CLICKHOUSE_REPO_PATH" ]; then
ls -lath ||:
fi
clickhouse_source="--clickhouse-source \'$CLICKHOUSE_PACKAGE\'"
clickhouse_source="--clickhouse-source $CLICKHOUSE_PACKAGE"
if [ -n "$WITH_LOCAL_BINARY" ]; then
clickhouse_source="--clickhouse-source /clickhouse"
fi
tests_count="--test-count \"$TESTS_TO_RUN\""
tests_count="--test-count $TESTS_TO_RUN"
tests_to_run="test-all"
workload=""
if [ -n "$WORKLOAD" ]; then

View File

@ -59,6 +59,12 @@ install_packages previous_release_package_folder
# available for dump via clickhouse-local
configure
# local_blob_storage disk type does not exist in older versions
sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \
| sed "s|<type>local_blob_storage</type>|<type>local</type>|" \
> /etc/clickhouse-server/config.d/storage_conf.xml.tmp
sudo mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml
start
stop
mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.initial.log
@ -83,6 +89,11 @@ export USE_S3_STORAGE_FOR_MERGE_TREE=1
export ZOOKEEPER_FAULT_INJECTION=0
configure
sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \
| sed "s|<type>local_blob_storage</type>|<type>local</type>|" \
> /etc/clickhouse-server/config.d/storage_conf.xml.tmp
sudo mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml
start
clickhouse-client --query="SELECT 'Server version: ', version()"

View File

@ -155,6 +155,9 @@ The following settings can be specified in configuration file for given endpoint
- `no_sign_request` - Ignore all the credentials so requests are not signed. Useful for accessing public buckets.
- `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be specified multiple times.
- `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. Optional.
- `server_side_encryption_kms_key_id` - If specified, required headers for accessing S3 objects with [SSE-KMS encryption](https://docs.aws.amazon.com/AmazonS3/latest/userguide/UsingKMSEncryption.html) will be set. If an empty string is specified, the AWS managed S3 key will be used. Optional.
- `server_side_encryption_kms_encryption_context` - If specified alongside `server_side_encryption_kms_key_id`, the given encryption context header for SSE-KMS will be set. Optional.
- `server_side_encryption_kms_bucket_key_enabled` - If specified alongside `server_side_encryption_kms_key_id`, the header to enable S3 bucket keys for SSE-KMS will be set. Optional, can be `true` or `false`, defaults to nothing (matches the bucket-level setting).
- `max_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. Optional.
- `max_put_rps`, `max_put_burst`, `max_get_rps` and `max_get_burst` - Throttling settings (see description above) to use for specific endpoint instead of per query. Optional.
@ -173,6 +176,9 @@ The following settings can be specified in configuration file for given endpoint
<!-- <no_sign_request>false</no_sign_request> -->
<!-- <header>Authorization: Bearer SOME-TOKEN</header> -->
<!-- <server_side_encryption_customer_key_base64>BASE64-ENCODED-KEY</server_side_encryption_customer_key_base64> -->
<!-- <server_side_encryption_kms_key_id>KMS_KEY_ID</server_side_encryption_kms_key_id> -->
<!-- <server_side_encryption_kms_encryption_context>KMS_ENCRYPTION_CONTEXT</server_side_encryption_kms_encryption_context> -->
<!-- <server_side_encryption_kms_bucket_key_enabled>true</server_side_encryption_kms_bucket_key_enabled> -->
<!-- <max_single_read_retries>4</max_single_read_retries> -->
</endpoint-name>
</s3>

View File

@ -1055,7 +1055,11 @@ Configuration markup:
<access_key_id>your_access_key_id</access_key_id>
<secret_access_key>your_secret_access_key</secret_access_key>
<region></region>
<header>Authorization: Bearer SOME-TOKEN</header>
<server_side_encryption_customer_key_base64>your_base64_encoded_customer_key</server_side_encryption_customer_key_base64>
<server_side_encryption_kms_key_id>your_kms_key_id</server_side_encryption_kms_key_id>
<server_side_encryption_kms_encryption_context>your_kms_encryption_context</server_side_encryption_kms_encryption_context>
<server_side_encryption_kms_bucket_key_enabled>true</server_side_encryption_kms_bucket_key_enabled>
<proxy>
<uri>http://proxy1</uri>
<uri>http://proxy2</uri>
@ -1106,7 +1110,11 @@ Optional parameters:
- `min_bytes_for_seek` — Minimal number of bytes to use seek operation instead of sequential read. Default value is `1 Mb`.
- `metadata_path` — Path on local FS to store metadata files for S3. Default value is `/var/lib/clickhouse/disks/<disk_name>/`.
- `skip_access_check` — If true, disk access checks will not be performed on disk start-up. Default value is `false`.
- `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be specified multiple times.
- `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set.
- `server_side_encryption_kms_key_id` - If specified, required headers for accessing S3 objects with [SSE-KMS encryption](https://docs.aws.amazon.com/AmazonS3/latest/userguide/UsingKMSEncryption.html) will be set. If an empty string is specified, the AWS managed S3 key will be used. Optional.
- `server_side_encryption_kms_encryption_context` - If specified alongside `server_side_encryption_kms_key_id`, the given encryption context header for SSE-KMS will be set. Optional.
- `server_side_encryption_kms_bucket_key_enabled` - If specified alongside `server_side_encryption_kms_key_id`, the header to enable S3 bucket keys for SSE-KMS will be set. Optional, can be `true` or `false`, defaults to nothing (matches the bucket-level setting).
- `s3_max_put_rps` — Maximum PUT requests per second rate before throttling. Default value is `0` (unlimited).
- `s3_max_put_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_put_rps`.
- `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited).

View File

@ -2068,3 +2068,20 @@ Possible values:
- Positive integer.
Default value: `10000`.
## display_secrets_in_show_and_select {#display_secrets_in_show_and_select}
Enables or disables showing secrets in `SHOW` and `SELECT` queries for tables, databases,
table functions, and dictionaries.
User wishing to see secrets must also have
[`format_display_secrets_in_show_and_select` format setting](../settings/formats#format_display_secrets_in_show_and_select)
turned on and a
[`displaySecretsInShowAndSelect`](../../sql-reference/statements/grant#grant-display-secrets) privilege.
Possible values:
- 0 — Disabled.
- 1 — Enabled.
Default value: 0.

View File

@ -7,6 +7,23 @@ toc_max_heading_level: 2
# Format settings {#format-settings}
## format_display_secrets_in_show_and_select {#format_display_secrets_in_show_and_select}
Enables or disables showing secrets in `SHOW` and `SELECT` queries for tables, databases,
table functions, and dictionaries.
User wishing to see secrets must also have
[`display_secrets_in_show_and_select` server setting](../server-configuration-parameters/settings#display_secrets_in_show_and_select)
turned on and a
[`displaySecretsInShowAndSelect`](../../sql-reference/statements/grant#grant-display-secrets) privilege.
Possible values:
- 0 — Disabled.
- 1 — Enabled.
Default value: 0.
## input_format_skip_unknown_fields {#input_format_skip_unknown_fields}
Enables or disables skipping insertion of extra data.

View File

@ -41,9 +41,9 @@ If the file is sitting on the same machine as `clickhouse-local`, use the `file`
```
ClickHouse knows the file uses a tab-separated format from filename extension. If you need to explicitly specify the format, simply add one of the [many ClickHouse input formats](../../interfaces/formats.md):
```bash
./clickhouse local -q "SELECT * FROM file('reviews.tsv', 'TabSeparated')"
```
```bash
./clickhouse local -q "SELECT * FROM file('reviews.tsv', 'TabSeparated')"
```
The `file` table function creates a table, and you can use `DESCRIBE` to see the inferred schema:

View File

@ -200,6 +200,7 @@ Hierarchy of privileges:
- `HDFS`
- `S3`
- [dictGet](#grant-dictget)
- [displaySecretsInShowAndSelect](#grant-display-secrets)
Examples of how this hierarchy is treated:
@ -485,6 +486,15 @@ Privilege level: `DICTIONARY`.
- `GRANT dictGet ON mydb.mydictionary TO john`
- `GRANT dictGet ON mydictionary TO john`
### displaySecretsInShowAndSelect {#grant-display-secrets}
Allows a user to view secrets in `SHOW` and `SELECT` queries if both
[`display_secrets_in_show_and_select` server setting](../../operations/server-configuration-parameters/settings#display_secrets_in_show_and_select)
and
[`format_display_secrets_in_show_and_select` format setting](../../operations/settings/formats#format_display_secrets_in_show_and_select)
are turned on.
### ALL
Grants all the privileges on regulated entity to a user account or a role.

View File

@ -6,6 +6,13 @@ sidebar_label: SHOW
# SHOW Statements
N.B. `SHOW CREATE (TABLE|DATABASE|USER)` hides secrets unless
[`display_secrets_in_show_and_select` server setting](../../operations/server-configuration-parameters/settings#display_secrets_in_show_and_select)
is turned on,
[`format_display_secrets_in_show_and_select` format setting](../../operations/settings/formats#format_display_secrets_in_show_and_select)
is turned on and user has
[`displaySecretsInShowAndSelect`](grant.md#grant-display-secrets) privilege.
## SHOW CREATE TABLE | DICTIONARY | VIEW | DATABASE
``` sql
@ -293,8 +300,6 @@ If user is not specified, the query returns privileges for the current user.
Shows parameters that were used at a [user creation](../../sql-reference/statements/create/user.md).
`SHOW CREATE USER` does not output user passwords.
**Syntax**
``` sql

View File

@ -135,7 +135,7 @@ func TestConfigFileFrameCopy(t *testing.T) {
sizes := map[string]int64{
"users.xml": int64(2017),
"default-password.xml": int64(188),
"config.xml": int64(61260),
"config.xml": int64(61662),
"server-include.xml": int64(168),
"user-include.xml": int64(559),
}

View File

@ -1260,8 +1260,12 @@
<access_key_id>REPLACE_ME</access_key_id>
<secret_access_key>REPLACE_ME</secret_access_key>
<region></region>
<header>Authorization: Bearer SOME-TOKEN</header>
<server_side_encryption_customer_key_base64>your_base64_encoded_customer_key
</server_side_encryption_customer_key_base64>
<server_side_encryption_kms_key_id>REPLACE_ME</server_side_encryption_kms_key_id>
<server_side_encryption_kms_encryption_context>REPLACE_ME</server_side_encryption_kms_encryption_context>
<server_side_encryption_kms_bucket_key_enabled>true</server_side_encryption_kms_bucket_key_enabled>
<proxy>
<uri>http://proxy1</uri>
<uri>http://proxy2</uri>

View File

@ -188,6 +188,7 @@ enum class AccessType
M(SYSTEM, "", GROUP, ALL) /* allows to execute SYSTEM {SHUTDOWN|RELOAD CONFIG|...} */ \
\
M(dictGet, "dictHas, dictGetHierarchy, dictIsIn", DICTIONARY, ALL) /* allows to execute functions dictGet(), dictHas(), dictGetHierarchy(), dictIsIn() */\
M(displaySecretsInShowAndSelect, "", GLOBAL, ALL) /* allows to show plaintext secrets in SELECT and SHOW queries. display_secrets_in_show_and_select format and server settings must be turned on */\
\
M(addressToLine, "", GLOBAL, INTROSPECTION) /* allows to execute function addressToLine() */\
M(addressToLineWithInlines, "", GLOBAL, INTROSPECTION) /* allows to execute function addressToLineWithInlines() */\

View File

@ -65,6 +65,7 @@ namespace
credentials.GetAWSAccessKeyId(),
credentials.GetAWSSecretKey(),
settings.auth_settings.server_side_encryption_customer_key_base64,
settings.auth_settings.server_side_encryption_kms_config,
std::move(headers),
S3::CredentialsConfiguration
{
@ -197,7 +198,7 @@ void BackupWriterS3::copyFileNative(DiskPtr src_disk, const String & src_file_na
auto object_storage = src_disk->getObjectStorage();
std::string src_bucket = object_storage->getObjectsNamespace();
auto file_path = fs::path(s3_uri.key) / dest_file_name;
copyS3File(client, src_bucket, objects[0].absolute_path, src_offset, src_size, s3_uri.bucket, file_path, request_settings, {},
copyS3File(client, src_bucket, objects[0].remote_path, src_offset, src_size, s3_uri.bucket, file_path, request_settings, {},
threadPoolCallbackRunner<void>(BackupsIOThreadPool::get(), "BackupWriterS3"));
}
}

View File

@ -136,6 +136,7 @@ if (TARGET ch_contrib::hdfs)
endif()
add_headers_and_sources(dbms Disks/ObjectStorages/Cached)
add_headers_and_sources(dbms Disks/ObjectStorages/Local)
add_headers_and_sources(dbms Disks/ObjectStorages/Web)
add_headers_and_sources(dbms Storages/Cache)

View File

@ -106,4 +106,23 @@ std::string_view CurrentThread::getQueryId()
return current_thread->getQueryId();
}
MemoryTracker * CurrentThread::getUserMemoryTracker()
{
if (unlikely(!current_thread))
return nullptr;
auto * tracker = current_thread->memory_tracker.getParent();
while (tracker && tracker->level != VariableContext::User)
tracker = tracker->getParent();
return tracker;
}
void CurrentThread::flushUntrackedMemory()
{
if (unlikely(!current_thread))
return;
current_thread->flushUntrackedMemory();
}
}

View File

@ -41,6 +41,12 @@ public:
/// Group to which belongs current thread
static ThreadGroupPtr getGroup();
/// MemoryTracker for user that owns current thread if any
static MemoryTracker * getUserMemoryTracker();
/// Adjust counters in MemoryTracker hierarchy if untracked_memory is not 0.
static void flushUntrackedMemory();
/// A logs queue used by TCPHandler to pass logs to a client
static void attachInternalTextLogsQueue(const std::shared_ptr<InternalTextLogsQueue> & logs_queue,
LogsLevel client_logs_level);

View File

@ -634,7 +634,6 @@
M(663, INCONSISTENT_METADATA_FOR_BACKUP) \
M(664, ACCESS_STORAGE_DOESNT_ALLOW_BACKUP) \
M(665, CANNOT_CONNECT_NATS) \
M(666, CANNOT_USE_CACHE) \
M(667, NOT_INITIALIZED) \
M(668, INVALID_STATE) \
M(669, NAMED_COLLECTION_DOESNT_EXIST) \

View File

@ -15,6 +15,7 @@
#include <Common/formatReadable.h>
#include <Common/filesystemHelpers.h>
#include <Common/ErrorCodes.h>
#include <Common/MemorySanitizer.h>
#include <Common/SensitiveDataMasker.h>
#include <Common/LockMemoryExceptionInThread.h>
#include <filesystem>
@ -96,7 +97,10 @@ Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc)
: Poco::Exception(exc.displayText(), ErrorCodes::POCO_EXCEPTION)
{
#ifdef STD_EXCEPTION_HAS_STACK_TRACE
set_stack_trace(exc.get_stack_trace_frames(), exc.get_stack_trace_size());
auto * stack_trace_frames = exc.get_stack_trace_frames();
auto stack_trace_size = exc.get_stack_trace_size();
__msan_unpoison(stack_trace_frames, stack_trace_size * sizeof(stack_trace_frames[0]));
set_stack_trace(stack_trace_frames, stack_trace_size);
#endif
}
@ -104,7 +108,10 @@ Exception::Exception(CreateFromSTDTag, const std::exception & exc)
: Poco::Exception(demangle(typeid(exc).name()) + ": " + String(exc.what()), ErrorCodes::STD_EXCEPTION)
{
#ifdef STD_EXCEPTION_HAS_STACK_TRACE
set_stack_trace(exc.get_stack_trace_frames(), exc.get_stack_trace_size());
auto * stack_trace_frames = exc.get_stack_trace_frames();
auto stack_trace_size = exc.get_stack_trace_size();
__msan_unpoison(stack_trace_frames, stack_trace_size * sizeof(stack_trace_frames[0]));
set_stack_trace(stack_trace_frames, stack_trace_size);
#endif
}
@ -112,7 +119,10 @@ Exception::Exception(CreateFromSTDTag, const std::exception & exc)
std::string getExceptionStackTraceString(const std::exception & e)
{
#ifdef STD_EXCEPTION_HAS_STACK_TRACE
return StackTrace::toString(e.get_stack_trace_frames(), 0, e.get_stack_trace_size());
auto * stack_trace_frames = e.get_stack_trace_frames();
auto stack_trace_size = e.get_stack_trace_size();
__msan_unpoison(stack_trace_frames, stack_trace_size * sizeof(stack_trace_frames[0]));
return StackTrace::toString(stack_trace_frames, 0, stack_trace_size);
#else
if (const auto * db_exception = dynamic_cast<const Exception *>(&e))
return db_exception->getStackTraceString();
@ -140,7 +150,10 @@ std::string getExceptionStackTraceString(std::exception_ptr e)
std::string Exception::getStackTraceString() const
{
#ifdef STD_EXCEPTION_HAS_STACK_TRACE
return StackTrace::toString(get_stack_trace_frames(), 0, get_stack_trace_size());
auto * stack_trace_frames = get_stack_trace_frames();
auto stack_trace_size = get_stack_trace_size();
__msan_unpoison(stack_trace_frames, stack_trace_size * sizeof(stack_trace_frames[0]));
return StackTrace::toString(stack_trace_frames, 0, stack_trace_size);
#else
return trace.toString();
#endif
@ -156,6 +169,7 @@ Exception::FramePointers Exception::getStackFramePointers() const
{
frame_pointers[i] = get_stack_trace_frames()[i];
}
__msan_unpoison(frame_pointers.data(), frame_pointers.size() * sizeof(frame_pointers[0]));
}
#else
{

View File

@ -30,10 +30,8 @@ static thread_local char thread_name[THREAD_NAME_SIZE]{};
void setThreadName(const char * name)
{
#ifndef NDEBUG
if (strlen(name) > THREAD_NAME_SIZE - 1)
throw DB::Exception(DB::ErrorCodes::PTHREAD_ERROR, "Thread name cannot be longer than 15 bytes");
#endif
#if defined(OS_FREEBSD)
pthread_set_name_np(pthread_self(), name);

View File

@ -102,6 +102,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo
credentials.GetAWSAccessKeyId(),
credentials.GetAWSSecretKey(),
auth_settings.server_side_encryption_customer_key_base64,
auth_settings.server_side_encryption_kms_config,
std::move(headers),
S3::CredentialsConfiguration
{

View File

@ -289,15 +289,20 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
nuraft::ptr<nuraft::buffer> latest_snapshot_ptr;
{ /// save snapshot into memory
std::lock_guard lock(snapshots_lock);
if (s.get_last_log_idx() != latest_snapshot_meta->get_last_log_idx())
if (s.get_last_log_idx() > latest_snapshot_meta->get_last_log_idx())
{
ProfileEvents::increment(ProfileEvents::KeeperSnapshotApplysFailed);
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Required to apply snapshot with last log index {}, but our last log index is {}",
"Required to apply snapshot with last log index {}, but last created snapshot was for smaller log index {}",
s.get_last_log_idx(),
latest_snapshot_meta->get_last_log_idx());
}
else if (s.get_last_log_idx() < latest_snapshot_meta->get_last_log_idx())
{
LOG_INFO(log, "A snapshot with a larger last log index ({}) was created, skipping applying this snapshot", latest_snapshot_meta->get_last_log_idx());
}
latest_snapshot_ptr = latest_snapshot_buf;
}
@ -372,19 +377,32 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res
{
{ /// Read storage data without locks and create snapshot
std::lock_guard lock(snapshots_lock);
auto [path, error_code] = snapshot_manager.serializeSnapshotToDisk(*snapshot);
if (error_code)
if (latest_snapshot_meta && snapshot->snapshot_meta->get_last_log_idx() <= latest_snapshot_meta->get_last_log_idx())
{
throw Exception(
ErrorCodes::SYSTEM_ERROR,
"Snapshot {} was created failed, error: {}",
LOG_INFO(
log,
"Will not create a snapshot with last log idx {} because a snapshot with bigger last log idx ({}) is already "
"created",
snapshot->snapshot_meta->get_last_log_idx(),
error_code.message());
latest_snapshot_meta->get_last_log_idx());
}
else
{
auto [path, error_code] = snapshot_manager.serializeSnapshotToDisk(*snapshot);
if (error_code)
{
throw Exception(
ErrorCodes::SYSTEM_ERROR,
"Snapshot {} was created failed, error: {}",
snapshot->snapshot_meta->get_last_log_idx(),
error_code.message());
}
latest_snapshot_path = path;
latest_snapshot_meta = snapshot->snapshot_meta;
ProfileEvents::increment(ProfileEvents::KeeperSnapshotCreations);
LOG_DEBUG(log, "Created persistent snapshot {} with path {}", latest_snapshot_meta->get_last_log_idx(), path);
}
latest_snapshot_path = path;
latest_snapshot_meta = snapshot->snapshot_meta;
ProfileEvents::increment(ProfileEvents::KeeperSnapshotCreations);
LOG_DEBUG(log, "Created persistent snapshot {} with path {}", latest_snapshot_meta->get_last_log_idx(), path);
}
{

View File

@ -76,6 +76,7 @@ namespace DB
M(UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \
M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \
M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \
M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0)
DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS)

View File

@ -663,7 +663,6 @@ class IColumn;
M(Bool, enable_filesystem_cache_on_write_operations, false, "Write into cache on write operations. To actually work this setting requires be added to disk config too", 0) \
M(Bool, enable_filesystem_cache_log, false, "Allows to record the filesystem caching log for each query", 0) \
M(Bool, read_from_filesystem_cache_if_exists_otherwise_bypass_cache, false, "Allow to use the filesystem cache in passive mode - benefit from the existing cache entries, but don't put more entries into the cache. If you set this setting for heavy ad-hoc queries and leave it disabled for short real-time queries, this will allows to avoid cache threshing by too heavy queries and to improve the overall system efficiency.", 0) \
M(Bool, enable_filesystem_cache_on_lower_level, true, "If read buffer supports caching inside threadpool, allow it to do it, otherwise cache outside ot threadpool. Do not use this setting, it is needed for testing", 0) \
M(Bool, skip_download_if_exceeds_query_cache, true, "Skip download from remote filesystem if exceeds query cache size", 0) \
M(UInt64, filesystem_cache_max_download_size, (128UL * 1024 * 1024 * 1024), "Max remote filesystem cache size that can be downloaded by a single query", 0) \
M(Bool, throw_on_error_from_cache_on_write_operations, false, "Ignore error from cache when caching on write operations (INSERT, merges)", 0) \
@ -903,7 +902,8 @@ class IColumn;
M(UInt64, output_format_pretty_max_value_width, 10000, "Maximum width of value to display in Pretty formats. If greater - it will be cut.", 0) \
M(Bool, output_format_pretty_color, true, "Use ANSI escape sequences to paint colors in Pretty formats", 0) \
M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \
M(UInt64, output_format_parquet_row_group_size, 1000000, "Row group size in rows.", 0) \
M(UInt64, output_format_parquet_row_group_size, 1000000, "Target row group size in rows.", 0) \
M(UInt64, output_format_parquet_row_group_size_bytes, 512 * 1024 * 1024, "Target row group size in bytes, before compression.", 0) \
M(Bool, output_format_parquet_string_as_string, false, "Use Parquet String type instead of Binary for String columns.", 0) \
M(Bool, output_format_parquet_fixed_string_as_fixed_byte_array, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary for FixedString columns.", 0) \
M(ParquetVersion, output_format_parquet_version, "2.latest", "Parquet format version for output format. Supported versions: 1.0, 2.4, 2.6 and 2.latest (default)", 0) \
@ -970,6 +970,7 @@ class IColumn;
M(Bool, input_format_bson_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format BSON.", 0) \
\
M(Bool, regexp_dict_allow_other_sources, false, "Allow regexp_tree dictionary to use sources other than yaml source.", 0) \
M(Bool, format_display_secrets_in_show_and_select, false, "Do not hide secrets in SHOW and SELECT queries.", IMPORTANT) \
M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \
\
M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \

View File

@ -9,9 +9,7 @@
#include <Common/quoteString.h>
#include <Common/atomicRename.h>
#include <Disks/IO/createReadBufferFromFileBase.h>
#include <Disks/ObjectStorages/LocalObjectStorage.h>
#include <Disks/ObjectStorages/DiskObjectStorage.h>
#include <Disks/ObjectStorages/FakeMetadataStorageFromDisk.h>
#include <Disks/loadLocalDiskConfig.h>
#include <Disks/TemporaryFileOnDisk.h>
#include <fstream>
@ -39,7 +37,6 @@ namespace DB
namespace ErrorCodes
{
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int PATH_ACCESS_DENIED;
extern const int LOGICAL_ERROR;
extern const int CANNOT_TRUNCATE_FILE;
@ -54,53 +51,6 @@ std::mutex DiskLocal::reservation_mutex;
using DiskLocalPtr = std::shared_ptr<DiskLocal>;
static void loadDiskLocalConfig(const String & name,
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
ContextPtr context,
String & path,
UInt64 & keep_free_space_bytes)
{
path = config.getString(config_prefix + ".path", "");
if (name == "default")
{
if (!path.empty())
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG,
"\"default\" disk path should be provided in <path> not it <storage_configuration>");
path = context->getPath();
}
else
{
if (path.empty())
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Disk path can not be empty. Disk {}", name);
if (path.back() != '/')
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Disk path must end with /. Disk {}", name);
if (path == context->getPath())
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Disk path ('{}') cannot be equal to <path>. Use <default> disk instead.", path);
}
bool has_space_ratio = config.has(config_prefix + ".keep_free_space_ratio");
if (config.has(config_prefix + ".keep_free_space_bytes") && has_space_ratio)
throw Exception(ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG,
"Only one of 'keep_free_space_bytes' and 'keep_free_space_ratio' can be specified");
keep_free_space_bytes = config.getUInt64(config_prefix + ".keep_free_space_bytes", 0);
if (has_space_ratio)
{
auto ratio = config.getDouble(config_prefix + ".keep_free_space_ratio");
if (ratio < 0 || ratio > 1)
throw Exception(ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG, "'keep_free_space_ratio' have to be between 0 and 1");
String tmp_path = path;
if (tmp_path.empty())
tmp_path = context->getPath();
// Create tmp disk for getting total disk space.
keep_free_space_bytes = static_cast<UInt64>(DiskLocal("tmp", tmp_path, 0).getTotalSpace() * ratio);
}
}
std::optional<size_t> fileSizeSafe(const fs::path & path)
{
std::error_code ec;
@ -604,25 +554,6 @@ catch (...)
return false;
}
DiskObjectStoragePtr DiskLocal::createDiskObjectStorage()
{
auto object_storage = std::make_shared<LocalObjectStorage>();
auto metadata_storage = std::make_shared<FakeMetadataStorageFromDisk>(
/* metadata_storage */std::static_pointer_cast<DiskLocal>(shared_from_this()),
object_storage,
/* object_storage_root_path */getPath());
return std::make_shared<DiskObjectStorage>(
getName(),
disk_path,
"Local",
metadata_storage,
object_storage,
false,
/* threadpool_size */16
);
}
void DiskLocal::checkAccessImpl(const String & path)
{
try
@ -750,13 +681,6 @@ void DiskLocal::chmod(const String & path, mode_t mode)
DB::throwFromErrnoWithPath("Cannot chmod file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED);
}
MetadataStoragePtr DiskLocal::getMetadataStorage()
{
auto object_storage = std::make_shared<LocalObjectStorage>();
return std::make_shared<FakeMetadataStorageFromDisk>(
std::static_pointer_cast<IDisk>(shared_from_this()), object_storage, getPath());
}
void registerDiskLocal(DiskFactory & factory, bool global_skip_access_check)
{
auto creator = [global_skip_access_check](

View File

@ -121,16 +121,12 @@ public:
bool canRead() const noexcept;
bool canWrite() noexcept;
DiskObjectStoragePtr createDiskObjectStorage() override;
bool supportsStat() const override { return true; }
struct stat stat(const String & path) const override;
bool supportsChmod() const override { return true; }
void chmod(const String & path, mode_t mode) override;
MetadataStoragePtr getMetadataStorage() override;
protected:
void checkAccessImpl(const String & path) override;

View File

@ -15,6 +15,7 @@ enum class DataSourceType
HDFS,
WebServer,
AzureBlobStorage,
LocalBlobStorage,
};
inline String toString(DataSourceType data_source_type)
@ -35,6 +36,8 @@ inline String toString(DataSourceType data_source_type)
return "web";
case DataSourceType::AzureBlobStorage:
return "azure_blob_storage";
case DataSourceType::LocalBlobStorage:
return "local_blob_storage";
}
UNREACHABLE();
}

View File

@ -7,9 +7,6 @@
#include <Common/logger_useful.h>
#include <Common/setThreadName.h>
#include <Core/ServerUUID.h>
#include <Disks/ObjectStorages/MetadataStorageFromDisk.h>
#include <Disks/ObjectStorages/FakeMetadataStorageFromDisk.h>
#include <Disks/ObjectStorages/LocalObjectStorage.h>
#include <Disks/FakeDiskTransaction.h>
namespace DB

View File

@ -367,7 +367,13 @@ public:
/// Actually it's a part of IDiskRemote implementation but we have so
/// complex hierarchy of disks (with decorators), so we cannot even
/// dynamic_cast some pointer to IDisk to pointer to IDiskRemote.
virtual MetadataStoragePtr getMetadataStorage() = 0;
virtual MetadataStoragePtr getMetadataStorage()
{
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Method getMetadataStorage() is not implemented for disk type: {}",
toString(getDataSourceDescription().type));
}
/// Very similar case as for getMetadataDiskIfExistsOrSelf(). If disk has "metadata"
/// it will return mapping for each required path: path -> metadata as string.

View File

@ -175,7 +175,7 @@ void AsynchronousReadIndirectBufferFromRemoteFS::appendToPrefetchLog(FilesystemP
{
.event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()),
.query_id = query_id,
.path = object.getMappedPath(),
.path = object.local_path,
.offset = file_offset_of_buffer_end,
.size = size,
.prefetch_submit_time = last_prefetch_info.submit_time,

View File

@ -5,6 +5,7 @@
#include <IO/ReadBufferFromFile.h>
#include <base/scope_guard.h>
#include <Common/assert_cast.h>
#include <IO/BoundedReadBuffer.h>
#include <Common/getRandomASCIIString.h>
#include <Common/logger_useful.h>
#include <base/hex.h>
@ -33,7 +34,6 @@ namespace DB
namespace ErrorCodes
{
extern const int CANNOT_SEEK_THROUGH_FILE;
extern const int CANNOT_USE_CACHE;
extern const int LOGICAL_ERROR;
extern const int ARGUMENT_OUT_OF_BOUND;
}
@ -190,12 +190,11 @@ CachedOnDiskReadBufferFromFile::getRemoteReadBuffer(FileSegment & file_segment,
if (!remote_fs_segment_reader)
{
remote_fs_segment_reader = implementation_buffer_creator();
if (!remote_fs_segment_reader->supportsRightBoundedReads())
throw Exception(
ErrorCodes::CANNOT_USE_CACHE,
"Cache cannot be used with a ReadBuffer which does not support right bounded reads");
auto impl = implementation_buffer_creator();
if (impl->supportsRightBoundedReads())
remote_fs_segment_reader = std::move(impl);
else
remote_fs_segment_reader = std::make_unique<BoundedReadBuffer>(std::move(impl));
file_segment.setRemoteFileReader(remote_fs_segment_reader);
}

View File

@ -20,8 +20,7 @@ namespace DB
class CachedOnDiskReadBufferFromFile : public ReadBufferFromFileBase
{
public:
using ImplementationBufferPtr = std::shared_ptr<ReadBufferFromFileBase>;
using ImplementationBufferCreator = std::function<ImplementationBufferPtr()>;
using ImplementationBufferCreator = std::function<std::unique_ptr<ReadBufferFromFileBase>()>;
CachedOnDiskReadBufferFromFile(
const String & source_file_path_,
@ -61,6 +60,8 @@ public:
};
private:
using ImplementationBufferPtr = std::shared_ptr<ReadBufferFromFileBase>;
void initialize(size_t offset, size_t size);
void assertCorrectness() const;

View File

@ -27,15 +27,11 @@ ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(
, read_buffer_creator(std::move(read_buffer_creator_))
, blobs_to_read(blobs_to_read_)
, settings(settings_)
, current_object(!blobs_to_read_.empty() ? blobs_to_read_.front() : throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read zero number of objects"))
, query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "")
, log(&Poco::Logger::get("ReadBufferFromRemoteFSGather"))
, enable_cache_log(!query_id.empty() && settings.enable_filesystem_cache_log)
{
if (blobs_to_read.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read zero number of objects");
current_object = blobs_to_read.front();
with_cache = settings.remote_fs_cache
&& settings.enable_filesystem_cache
&& (!query_id.empty() || settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache);
@ -50,7 +46,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c
current_object = object;
total_bytes_read_from_current_file = 0;
const auto & object_path = object.absolute_path;
const auto & object_path = object.remote_path;
size_t current_read_until_position = read_until_position ? read_until_position : object.bytes_size;
auto current_read_buffer_creator = [=, this]() { return read_buffer_creator(object_path, current_read_until_position); };
@ -76,12 +72,12 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c
void ReadBufferFromRemoteFSGather::appendFilesystemCacheLog()
{
chassert(!current_object.absolute_path.empty());
chassert(!current_object.remote_path.empty());
FilesystemCacheLogElement elem
{
.event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()),
.query_id = query_id,
.source_file_path = current_object.absolute_path,
.source_file_path = current_object.remote_path,
.file_segment_range = { 0, current_object.bytes_size },
.cache_type = FilesystemCacheLogElement::CacheType::READ_FROM_FS_BYPASSING_CACHE,
.file_segment_size = total_bytes_read_from_current_file,
@ -123,6 +119,8 @@ void ReadBufferFromRemoteFSGather::initialize()
if (object.bytes_size > current_buf_offset)
{
LOG_TEST(log, "Reading from file: {} ({})", object.remote_path, object.local_path);
/// Do not create a new buffer if we already have what we need.
if (!current_buf || current_buf_idx != i)
{
@ -174,6 +172,7 @@ bool ReadBufferFromRemoteFSGather::moveToNextBuffer()
++current_buf_idx;
const auto & object = blobs_to_read[current_buf_idx];
LOG_TEST(log, "Reading from next file: {} ({})", object.remote_path, object.local_path);
current_buf = createImplementationBuffer(object);
return true;
@ -246,7 +245,7 @@ void ReadBufferFromRemoteFSGather::reset()
String ReadBufferFromRemoteFSGather::getFileName() const
{
return current_object.absolute_path;
return current_object.remote_path;
}
size_t ReadBufferFromRemoteFSGather::getFileSize() const

View File

@ -20,7 +20,7 @@ class ReadBufferFromRemoteFSGather final : public ReadBuffer
friend class ReadIndirectBufferFromRemoteFS;
public:
using ReadBufferCreator = std::function<std::shared_ptr<ReadBufferFromFileBase>(const std::string & path, size_t read_until_position)>;
using ReadBufferCreator = std::function<std::unique_ptr<ReadBufferFromFileBase>(const std::string & path, size_t read_until_position)>;
ReadBufferFromRemoteFSGather(
ReadBufferCreator && read_buffer_creator_,

View File

@ -50,7 +50,7 @@ bool AzureObjectStorage::exists(const StoredObject & object) const
/// What a shame, no Exists method...
Azure::Storage::Blobs::ListBlobsOptions options;
options.Prefix = object.absolute_path;
options.Prefix = object.remote_path;
options.PageSizeHint = 1;
auto blobs_list_response = client_ptr->ListBlobs(options);
@ -58,7 +58,7 @@ bool AzureObjectStorage::exists(const StoredObject & object) const
for (const auto & blob : blobs_list)
{
if (object.absolute_path == blob.Name)
if (object.remote_path == blob.Name)
return true;
}
@ -74,7 +74,7 @@ std::unique_ptr<ReadBufferFromFileBase> AzureObjectStorage::readObject( /// NOLI
auto settings_ptr = settings.get();
return std::make_unique<ReadBufferFromAzureBlobStorage>(
client.get(), object.absolute_path, patchSettings(read_settings), settings_ptr->max_single_read_retries,
client.get(), object.remote_path, patchSettings(read_settings), settings_ptr->max_single_read_retries,
settings_ptr->max_single_download_retries);
}
@ -89,7 +89,7 @@ std::unique_ptr<ReadBufferFromFileBase> AzureObjectStorage::readObjects( /// NOL
auto read_buffer_creator =
[this, settings_ptr, disk_read_settings]
(const std::string & path, size_t read_until_position) -> std::shared_ptr<ReadBufferFromFileBase>
(const std::string & path, size_t read_until_position) -> std::unique_ptr<ReadBufferFromFileBase>
{
return std::make_unique<ReadBufferFromAzureBlobStorage>(
client.get(),
@ -130,16 +130,16 @@ std::unique_ptr<WriteBufferFromFileBase> AzureObjectStorage::writeObject( /// NO
if (mode != WriteMode::Rewrite)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Azure storage doesn't support append");
LOG_TEST(log, "Writing file: {}", object.absolute_path);
LOG_TEST(log, "Writing file: {}", object.remote_path);
auto buffer = std::make_unique<WriteBufferFromAzureBlobStorage>(
client.get(),
object.absolute_path,
object.remote_path,
settings.get()->max_single_part_upload_size,
buf_size,
patchSettings(write_settings));
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(buffer), std::move(finalize_callback), object.absolute_path);
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(buffer), std::move(finalize_callback), object.remote_path);
}
void AzureObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const
@ -172,7 +172,7 @@ void AzureObjectStorage::findAllFiles(const std::string & path, RelativePathsWit
/// Remove file. Throws exception if file doesn't exists or it's a directory.
void AzureObjectStorage::removeObject(const StoredObject & object)
{
const auto & path = object.absolute_path;
const auto & path = object.remote_path;
LOG_TEST(log, "Removing single object: {}", path);
auto client_ptr = client.get();
auto delete_info = client_ptr->DeleteBlob(path);
@ -187,12 +187,12 @@ void AzureObjectStorage::removeObjects(const StoredObjects & objects)
auto client_ptr = client.get();
for (const auto & object : objects)
{
LOG_TEST(log, "Removing object: {} (total: {})", object.absolute_path, objects.size());
auto delete_info = client_ptr->DeleteBlob(object.absolute_path);
LOG_TEST(log, "Removing object: {} (total: {})", object.remote_path, objects.size());
auto delete_info = client_ptr->DeleteBlob(object.remote_path);
if (!delete_info.Value.Deleted)
throw Exception(
ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Failed to delete file (path: {}) in AzureBlob Storage, reason: {}",
object.absolute_path, delete_info.RawResponse ? delete_info.RawResponse->GetReasonPhrase() : "Unknown");
object.remote_path, delete_info.RawResponse ? delete_info.RawResponse->GetReasonPhrase() : "Unknown");
}
}
@ -201,8 +201,8 @@ void AzureObjectStorage::removeObjectIfExists(const StoredObject & object)
auto client_ptr = client.get();
try
{
LOG_TEST(log, "Removing single object: {}", object.absolute_path);
auto delete_info = client_ptr->DeleteBlob(object.absolute_path);
LOG_TEST(log, "Removing single object: {}", object.remote_path);
auto delete_info = client_ptr->DeleteBlob(object.remote_path);
}
catch (const Azure::Storage::StorageException & e)
{
@ -221,7 +221,7 @@ void AzureObjectStorage::removeObjectsIfExist(const StoredObjects & objects)
{
try
{
auto delete_info = client_ptr->DeleteBlob(object.absolute_path);
auto delete_info = client_ptr->DeleteBlob(object.remote_path);
}
catch (const Azure::Storage::StorageException & e)
{
@ -259,8 +259,8 @@ void AzureObjectStorage::copyObject( /// NOLINT
std::optional<ObjectAttributes> object_to_attributes)
{
auto client_ptr = client.get();
auto dest_blob_client = client_ptr->GetBlobClient(object_to.absolute_path);
auto source_blob_client = client_ptr->GetBlobClient(object_from.absolute_path);
auto dest_blob_client = client_ptr->GetBlobClient(object_to.remote_path);
auto source_blob_client = client_ptr->GetBlobClient(object_from.remote_path);
Azure::Storage::Blobs::CopyBlobFromUriOptions copy_options;
if (object_to_attributes.has_value())

View File

@ -59,7 +59,7 @@ ReadSettings CachedObjectStorage::patchSettings(const ReadSettings & read_settin
if (!canUseReadThroughCache())
modified_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true;
return IObjectStorage::patchSettings(modified_settings);
return object_storage->patchSettings(modified_settings);
}
void CachedObjectStorage::startup()
@ -80,45 +80,7 @@ std::unique_ptr<ReadBufferFromFileBase> CachedObjectStorage::readObjects( /// NO
{
if (objects.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Received empty list of objects to read");
assert(!objects[0].getPathKeyForCache().empty());
/// Add cache relating settings to ReadSettings.
auto modified_read_settings = patchSettings(read_settings);
auto implementation_buffer = object_storage->readObjects(objects, modified_read_settings, read_hint, file_size);
/// If underlying read buffer does caching on its own, do not wrap it in caching buffer.
if (implementation_buffer->isIntegratedWithFilesystemCache()
&& modified_read_settings.enable_filesystem_cache_on_lower_level)
{
return implementation_buffer;
}
else
{
if (!file_size)
file_size = implementation_buffer->getFileSize();
auto implementation_buffer_creator = [objects, modified_read_settings, read_hint, file_size, this]()
{
return std::make_unique<BoundedReadBuffer>(
object_storage->readObjects(objects, modified_read_settings, read_hint, file_size));
};
/// TODO: A test is needed for the case of non-s3 storage and *Log family engines.
std::string path = objects[0].absolute_path;
FileCache::Key key = getCacheKey(objects[0].getPathKeyForCache());
return std::make_unique<CachedOnDiskReadBufferFromFile>(
path,
key,
cache,
implementation_buffer_creator,
modified_read_settings,
CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() ? std::string(CurrentThread::getQueryId()) : "",
file_size.value(),
/* allow_seeks */true,
/* use_external_buffer */false);
}
return object_storage->readObjects(objects, patchSettings(read_settings), read_hint, file_size);
}
std::unique_ptr<ReadBufferFromFileBase> CachedObjectStorage::readObject( /// NOLINT
@ -127,42 +89,9 @@ std::unique_ptr<ReadBufferFromFileBase> CachedObjectStorage::readObject( /// NOL
std::optional<size_t> read_hint,
std::optional<size_t> file_size) const
{
/// Add cache relating settings to ReadSettings.
auto modified_read_settings = patchSettings(read_settings);
auto implementation_buffer = object_storage->readObject(object, read_settings, read_hint, file_size);
/// If underlying read buffer does caching on its own, do not wrap it in caching buffer.
if (implementation_buffer->isIntegratedWithFilesystemCache()
&& modified_read_settings.enable_filesystem_cache_on_lower_level)
{
return implementation_buffer;
}
else
{
if (!file_size)
file_size = implementation_buffer->getFileSize();
auto implementation_buffer_creator = [object, read_settings, read_hint, file_size, this]()
{
return std::make_unique<BoundedReadBuffer>(object_storage->readObject(object, read_settings, read_hint, file_size));
};
FileCache::Key key = getCacheKey(object.getPathKeyForCache());
LOG_TEST(log, "Reading from file `{}` with cache key `{}`", object.absolute_path, key.toString());
return std::make_unique<CachedOnDiskReadBufferFromFile>(
object.absolute_path,
key,
cache,
implementation_buffer_creator,
read_settings,
CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() ? std::string(CurrentThread::getQueryId()) : "",
file_size.value(),
/* allow_seeks */true,
/* use_external_buffer */false);
}
return object_storage->readObject(object, patchSettings(read_settings), read_hint, file_size);
}
std::unique_ptr<WriteBufferFromFileBase> CachedObjectStorage::writeObject( /// NOLINT
const StoredObject & object,
WriteMode mode, // Cached doesn't support append, only rewrite
@ -177,16 +106,14 @@ std::unique_ptr<WriteBufferFromFileBase> CachedObjectStorage::writeObject( /// N
bool cache_on_write = modified_write_settings.enable_filesystem_cache_on_write_operations
&& FileCacheFactory::instance().getByName(cache_config_name).settings.cache_on_write_operations
&& fs::path(object.absolute_path).extension() != ".tmp";
&& fs::path(object.remote_path).extension() != ".tmp";
auto path_key_for_cache = object.getPathKeyForCache();
/// Need to remove even if cache_on_write == false.
removeCacheIfExists(path_key_for_cache);
removeCacheIfExists(object.remote_path);
if (cache_on_write)
{
auto key = getCacheKey(path_key_for_cache);
auto key = getCacheKey(object.remote_path);
return std::make_unique<CachedOnDiskWriteBufferFromFile>(
std::move(implementation_buffer),
cache,
@ -211,28 +138,27 @@ void CachedObjectStorage::removeCacheIfExists(const std::string & path_key_for_c
void CachedObjectStorage::removeObject(const StoredObject & object)
{
removeCacheIfExists(object.getPathKeyForCache());
object_storage->removeObject(object);
}
void CachedObjectStorage::removeObjects(const StoredObjects & objects)
{
for (const auto & object : objects)
removeCacheIfExists(object.getPathKeyForCache());
removeCacheIfExists(object.remote_path);
object_storage->removeObjects(objects);
}
void CachedObjectStorage::removeObjectIfExists(const StoredObject & object)
{
removeCacheIfExists(object.getPathKeyForCache());
removeCacheIfExists(object.remote_path);
object_storage->removeObjectIfExists(object);
}
void CachedObjectStorage::removeObjectsIfExist(const StoredObjects & objects)
{
for (const auto & object : objects)
removeCacheIfExists(object.getPathKeyForCache());
removeCacheIfExists(object.remote_path);
object_storage->removeObjectsIfExist(objects);
}

View File

@ -47,6 +47,9 @@ void registerDiskCache(DiskFactory & factory, bool /* global_skip_access_check *
auto cache = FileCacheFactory::instance().getOrCreate(name, file_cache_settings);
auto disk = disk_it->second;
if (!dynamic_cast<const DiskObjectStorage *>(disk.get()))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cached disk is allowed only on top of object storage");
auto disk_object_storage = disk->createDiskObjectStorage();
disk_object_storage->wrapWithCache(cache, file_cache_settings, name);

View File

@ -297,7 +297,7 @@ String DiskObjectStorage::getUniqueId(const String & path) const
String id;
auto blobs_paths = metadata_storage->getStorageObjects(path);
if (!blobs_paths.empty())
id = blobs_paths[0].absolute_path;
id = blobs_paths[0].remote_path;
return id;
}
@ -309,7 +309,7 @@ bool DiskObjectStorage::checkUniqueId(const String & id) const
return false;
}
auto object = StoredObject::create(*object_storage, id, {}, {}, true);
auto object = StoredObject(id);
return object_storage->exists(object);
}

View File

@ -98,7 +98,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::migrateFileToRestorableSchema
ObjectAttributes metadata {
{"path", path}
};
updateObjectMetadata(object.absolute_path, metadata);
updateObjectMetadata(object.remote_path, metadata);
}
}
void DiskObjectStorageRemoteMetadataRestoreHelper::migrateToRestorableSchemaRecursive(const String & path, Futures & results)

View File

@ -109,7 +109,7 @@ struct RemoveObjectStorageOperation final : public IDiskObjectStorageOperation
if (hardlink_count == 0)
{
objects_to_remove = objects;
objects_to_remove = std::move(objects);
}
}
catch (const Exception & e)
@ -194,7 +194,7 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati
/// File is really redundant
if (hardlink_count == 0 && !keep_all_batch_data && !file_names_remove_metadata_only.contains(fs::path(path).filename()))
objects_to_remove.insert(objects_to_remove.end(), objects.begin(), objects.end());
std::move(objects.begin(), objects.end(), std::back_inserter(objects_to_remove));
}
catch (const Exception & e)
{
@ -267,7 +267,7 @@ struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOp
if (hardlink_count == 0)
{
objects_to_remove[path_to_remove] = objects_paths;
objects_to_remove[path_to_remove] = std::move(objects_paths);
}
}
catch (const Exception & e)
@ -321,7 +321,7 @@ struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOp
{
if (!file_names_remove_metadata_only.contains(fs::path(local_path).filename()))
{
remove_from_remote.insert(remove_from_remote.end(), remote_paths.begin(), remote_paths.end());
std::move(remote_paths.begin(), remote_paths.end(), std::back_inserter(remove_from_remote));
}
}
/// Read comment inside RemoveObjectStorageOperation class
@ -450,8 +450,7 @@ struct CopyFileObjectStorageOperation final : public IDiskObjectStorageOperation
for (const auto & object_from : source_blobs)
{
std::string blob_name = object_storage.generateBlobNameForPath(to_path);
auto object_to = StoredObject::create(
object_storage, fs::path(metadata_storage.getObjectStorageRootPath()) / blob_name);
auto object_to = StoredObject(fs::path(metadata_storage.getObjectStorageRootPath()) / blob_name);
object_storage.copyObject(object_from, object_to);
@ -616,7 +615,7 @@ std::unique_ptr<WriteBufferFromFileBase> DiskObjectStorageTransaction::writeFile
blob_name = "r" + revisionToString(revision) + "-file-" + blob_name;
}
auto object = StoredObject::create(object_storage, fs::path(metadata_storage.getObjectStorageRootPath()) / blob_name);
auto object = StoredObject(fs::path(metadata_storage.getObjectStorageRootPath()) / blob_name);
auto write_operation = std::make_unique<WriteFileObjectStorageOperation>(object_storage, metadata_storage, object);
std::function<void(size_t count)> create_metadata_callback;
@ -690,7 +689,7 @@ void DiskObjectStorageTransaction::writeFileUsingCustomWriteObject(
blob_name = "r" + revisionToString(revision) + "-file-" + blob_name;
}
auto object = StoredObject::create(object_storage, fs::path(metadata_storage.getObjectStorageRootPath()) / blob_name);
auto object = StoredObject(fs::path(metadata_storage.getObjectStorageRootPath()) / blob_name);
auto write_operation = std::make_unique<WriteFileObjectStorageOperation>(object_storage, metadata_storage, object);
operations_to_execute.emplace_back(std::move(write_operation));

View File

@ -1,213 +0,0 @@
#include "FakeMetadataStorageFromDisk.h"
#include <Disks/IDisk.h>
#include <Common/filesystemHelpers.h>
#include <Common/logger_useful.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
FakeMetadataStorageFromDisk::FakeMetadataStorageFromDisk(
DiskPtr disk_,
ObjectStoragePtr object_storage_,
const std::string & object_storage_root_path_)
: disk(disk_)
, object_storage(object_storage_)
, object_storage_root_path(object_storage_root_path_)
{
}
MetadataTransactionPtr FakeMetadataStorageFromDisk::createTransaction()
{
return std::make_shared<FakeMetadataStorageFromDiskTransaction>(*this, disk);
}
const std::string & FakeMetadataStorageFromDisk::getPath() const
{
return disk->getPath();
}
bool FakeMetadataStorageFromDisk::exists(const std::string & path) const
{
return disk->exists(path);
}
bool FakeMetadataStorageFromDisk::isFile(const std::string & path) const
{
return disk->isFile(path);
}
bool FakeMetadataStorageFromDisk::isDirectory(const std::string & path) const
{
return disk->isDirectory(path);
}
Poco::Timestamp FakeMetadataStorageFromDisk::getLastModified(const std::string & path) const
{
return disk->getLastModified(path);
}
time_t FakeMetadataStorageFromDisk::getLastChanged(const std::string & path) const
{
return disk->getLastChanged(path);
}
uint64_t FakeMetadataStorageFromDisk::getFileSize(const String & path) const
{
return disk->getFileSize(path);
}
std::vector<std::string> FakeMetadataStorageFromDisk::listDirectory(const std::string & path) const
{
std::vector<std::string> result;
disk->listFiles(path, result);
return result;
}
DirectoryIteratorPtr FakeMetadataStorageFromDisk::iterateDirectory(const std::string & path) const
{
return disk->iterateDirectory(path);
}
std::string FakeMetadataStorageFromDisk::readFileToString(const std::string &) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "readFileToString is not implemented for FakeMetadataStorageFromDisk");
}
std::string FakeMetadataStorageFromDisk::readInlineDataToString(const std::string & path) const
{
auto rb = disk->readFile(path);
std::string result;
std::array<char, 1000> buf;
while (!rb->eof())
{
auto sz = rb->read(buf.data(), buf.size());
result.append(buf.data(), buf.data() + sz);
}
return result;
}
std::unordered_map<String, String> FakeMetadataStorageFromDisk::getSerializedMetadata(const std::vector<String> &) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getSerializedMetadata is not implemented for FakeMetadataStorageFromDisk");
}
StoredObjects FakeMetadataStorageFromDisk::getStorageObjects(const std::string & path) const
{
std::string blob_name = object_storage->generateBlobNameForPath(path);
std::string object_path = fs::path(object_storage_root_path) / blob_name;
size_t object_size = getFileSize(path);
auto object = StoredObject::create(*object_storage, object_path, object_size, path, /* exists */true);
return {std::move(object)};
}
uint32_t FakeMetadataStorageFromDisk::getHardlinkCount(const std::string & path) const
{
return disk->getRefCount(path);
}
const IMetadataStorage & FakeMetadataStorageFromDiskTransaction::getStorageForNonTransactionalReads() const
{
return metadata_storage;
}
void FakeMetadataStorageFromDiskTransaction::writeStringToFile(const std::string & path, const std::string & data)
{
auto wb = disk->writeFile(path);
wb->write(data.data(), data.size());
wb->finalize();
}
void FakeMetadataStorageFromDiskTransaction::writeInlineDataToFile(const std::string & path, const std::string & data)
{
auto wb = disk->writeFile(path);
wb->write(data.data(), data.size());
wb->finalize();
}
void FakeMetadataStorageFromDiskTransaction::setLastModified(const std::string & path, const Poco::Timestamp & timestamp)
{
disk->setLastModified(path, timestamp);
}
void FakeMetadataStorageFromDiskTransaction::unlinkFile(const std::string & path)
{
disk->removeFile(path);
}
void FakeMetadataStorageFromDiskTransaction::removeRecursive(const std::string & path)
{
disk->removeRecursive(path);
}
void FakeMetadataStorageFromDiskTransaction::createDirectory(const std::string & path)
{
disk->createDirectory(path);
}
void FakeMetadataStorageFromDiskTransaction::createDirectoryRecursive(const std::string & path)
{
disk->createDirectories(path);
}
void FakeMetadataStorageFromDiskTransaction::removeDirectory(const std::string & path)
{
disk->removeDirectory(path);
}
void FakeMetadataStorageFromDiskTransaction::moveFile(const std::string & path_from, const std::string & path_to)
{
disk->moveFile(path_from, path_to);
}
void FakeMetadataStorageFromDiskTransaction::moveDirectory(const std::string & path_from, const std::string & path_to)
{
disk->moveDirectory(path_from, path_to);
}
void FakeMetadataStorageFromDiskTransaction::replaceFile(const std::string & path_from, const std::string & path_to)
{
disk->replaceFile(path_from, path_to);
}
void FakeMetadataStorageFromDiskTransaction::setReadOnly(const std::string & path)
{
disk->setReadOnly(path);
}
void FakeMetadataStorageFromDiskTransaction::createHardLink(const std::string & path_from, const std::string & path_to)
{
disk->createHardLink(path_from, path_to);
}
void FakeMetadataStorageFromDiskTransaction::createEmptyMetadataFile(const std::string & /* path */)
{
/// Noop.
}
void FakeMetadataStorageFromDiskTransaction::createMetadataFile(
const std::string & /* path */, const std::string & /* blob_name */, uint64_t /* size_in_bytes */)
{
/// Noop.
}
void FakeMetadataStorageFromDiskTransaction::addBlobToMetadata(
const std::string & /* path */, const std::string & /* blob_name */, uint64_t /* size_in_bytes */)
{
/// Noop, local metadata files is only one file, it is the metadata file itself.
}
void FakeMetadataStorageFromDiskTransaction::unlinkMetadata(const std::string & path)
{
disk->removeFile(path);
}
}

View File

@ -1,131 +0,0 @@
#pragma once
#include <Common/SharedMutex.h>
#include <Disks/IDisk.h>
#include <Disks/ObjectStorages/IMetadataStorage.h>
#include <Disks/ObjectStorages/MetadataFromDiskTransactionState.h>
#include <Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h>
namespace DB
{
/// Store metadata in the disk itself.
class FakeMetadataStorageFromDisk final : public IMetadataStorage
{
private:
friend class FakeMetadataStorageFromDiskTransaction;
mutable SharedMutex metadata_mutex;
DiskPtr disk;
ObjectStoragePtr object_storage;
std::string object_storage_root_path;
public:
FakeMetadataStorageFromDisk(
DiskPtr disk_,
ObjectStoragePtr object_storage_,
const std::string & object_storage_root_path_);
MetadataTransactionPtr createTransaction() override;
const std::string & getPath() const override;
bool exists(const std::string & path) const override;
bool isFile(const std::string & path) const override;
bool isDirectory(const std::string & path) const override;
uint64_t getFileSize(const String & path) const override;
Poco::Timestamp getLastModified(const std::string & path) const override;
time_t getLastChanged(const std::string & path) const override;
bool supportsChmod() const override { return disk->supportsChmod(); }
bool supportsStat() const override { return disk->supportsStat(); }
struct stat stat(const String & path) const override { return disk->stat(path); }
std::vector<std::string> listDirectory(const std::string & path) const override;
DirectoryIteratorPtr iterateDirectory(const std::string & path) const override;
std::string readFileToString(const std::string & path) const override;
std::string readInlineDataToString(const std::string & path) const override;
std::unordered_map<String, String> getSerializedMetadata(const std::vector<String> & file_paths) const override;
uint32_t getHardlinkCount(const std::string & path) const override;
DiskPtr getDisk() const { return disk; }
StoredObjects getStorageObjects(const std::string & path) const override;
std::string getObjectStorageRootPath() const override { return object_storage_root_path; }
};
class FakeMetadataStorageFromDiskTransaction final : public IMetadataTransaction
{
private:
DiskPtr disk;
const FakeMetadataStorageFromDisk & metadata_storage;
std::vector<MetadataOperationPtr> operations;
public:
FakeMetadataStorageFromDiskTransaction(
const FakeMetadataStorageFromDisk & metadata_storage_, DiskPtr disk_)
: disk(disk_)
, metadata_storage(metadata_storage_)
{}
~FakeMetadataStorageFromDiskTransaction() override = default;
const IMetadataStorage & getStorageForNonTransactionalReads() const final;
void commit() final {}
void writeStringToFile(const std::string & path, const std::string & data) override;
void writeInlineDataToFile(const std::string & path, const std::string & data) override;
void createEmptyMetadataFile(const std::string & path) override;
void createMetadataFile(const std::string & path, const std::string & blob_name, uint64_t size_in_bytes) override;
void addBlobToMetadata(const std::string & path, const std::string & blob_name, uint64_t size_in_bytes) override;
void setLastModified(const std::string & path, const Poco::Timestamp & timestamp) override;
bool supportsChmod() const override { return disk->supportsChmod(); }
void chmod(const String & path, mode_t mode) override { disk->chmod(path, mode); }
void setReadOnly(const std::string & path) override;
void unlinkFile(const std::string & path) override;
void createDirectory(const std::string & path) override;
void createDirectoryRecursive(const std::string & path) override;
void removeDirectory(const std::string & path) override;
void removeRecursive(const std::string & path) override;
void createHardLink(const std::string & path_from, const std::string & path_to) override;
void moveFile(const std::string & path_from, const std::string & path_to) override;
void moveDirectory(const std::string & path_from, const std::string & path_to) override;
void replaceFile(const std::string & path_from, const std::string & path_to) override;
void unlinkMetadata(const std::string & path) override;
};
}

View File

@ -40,7 +40,7 @@ std::string HDFSObjectStorage::generateBlobNameForPath(const std::string & /* pa
bool HDFSObjectStorage::exists(const StoredObject & object) const
{
const auto & path = object.absolute_path;
const auto & path = object.remote_path;
const size_t begin_of_path = path.find('/', path.find("//") + 2);
const String remote_fs_object_path = path.substr(begin_of_path);
return (0 == hdfsExists(hdfs_fs.get(), remote_fs_object_path.c_str()));
@ -52,7 +52,7 @@ std::unique_ptr<ReadBufferFromFileBase> HDFSObjectStorage::readObject( /// NOLIN
std::optional<size_t>,
std::optional<size_t>) const
{
return std::make_unique<ReadBufferFromHDFS>(object.absolute_path, object.absolute_path, config, patchSettings(read_settings));
return std::make_unique<ReadBufferFromHDFS>(object.remote_path, object.remote_path, config, patchSettings(read_settings));
}
std::unique_ptr<ReadBufferFromFileBase> HDFSObjectStorage::readObjects( /// NOLINT
@ -64,7 +64,7 @@ std::unique_ptr<ReadBufferFromFileBase> HDFSObjectStorage::readObjects( /// NOLI
auto disk_read_settings = patchSettings(read_settings);
auto read_buffer_creator =
[this, disk_read_settings]
(const std::string & path, size_t /* read_until_position */) -> std::shared_ptr<ReadBufferFromFileBase>
(const std::string & path, size_t /* read_until_position */) -> std::unique_ptr<ReadBufferFromFileBase>
{
size_t begin_of_path = path.find('/', path.find("//") + 2);
auto hdfs_path = path.substr(begin_of_path);
@ -94,17 +94,17 @@ std::unique_ptr<WriteBufferFromFileBase> HDFSObjectStorage::writeObject( /// NOL
/// Single O_WRONLY in libhdfs adds O_TRUNC
auto hdfs_buffer = std::make_unique<WriteBufferFromHDFS>(
object.absolute_path, config, settings->replication, patchSettings(write_settings), buf_size,
object.remote_path, config, settings->replication, patchSettings(write_settings), buf_size,
mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND);
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(hdfs_buffer), std::move(finalize_callback), object.absolute_path);
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(hdfs_buffer), std::move(finalize_callback), object.remote_path);
}
/// Remove file. Throws exception if file doesn't exists or it's a directory.
void HDFSObjectStorage::removeObject(const StoredObject & object)
{
const auto & path = object.absolute_path;
const auto & path = object.remote_path;
const size_t begin_of_path = path.find('/', path.find("//") + 2);
/// Add path from root to file name

View File

@ -1,5 +1,6 @@
#include <Disks/ObjectStorages/IObjectStorage.h>
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
#include <Common/getRandomASCIIString.h>
#include <IO/WriteBufferFromFileBase.h>
#include <IO/copyData.h>
#include <Interpreters/Context.h>
@ -88,4 +89,21 @@ WriteSettings IObjectStorage::patchSettings(const WriteSettings & write_settings
return settings;
}
std::string IObjectStorage::generateBlobNameForPath(const std::string & /* path */)
{
/// Path to store the new S3 object.
/// Total length is 32 a-z characters for enough randomness.
/// First 3 characters are used as a prefix for
/// https://aws.amazon.com/premiumsupport/knowledge-center/s3-object-key-naming-pattern/
constexpr size_t key_name_total_size = 32;
constexpr size_t key_name_prefix_size = 3;
/// Path to store new S3 object.
return fmt::format("{}/{}",
getRandomASCIIString(key_name_prefix_size),
getRandomASCIIString(key_name_total_size - key_name_prefix_size));
}
}

View File

@ -184,15 +184,12 @@ public:
/// Generate blob name for passed absolute local path.
/// Path can be generated either independently or based on `path`.
virtual std::string generateBlobNameForPath(const std::string & path) = 0;
virtual std::string generateBlobNameForPath(const std::string & path);
/// Get unique id for passed absolute path in object storage.
virtual std::string getUniqueId(const std::string & path) const { return path; }
virtual bool supportsAppend() const { return false; }
/// Remove filesystem cache. `path` is a result of object.getPathKeyForCache() method,
/// which is used to define a cache key for the source object path.
/// Remove filesystem cache.
virtual void removeCacheIfExists(const std::string & /* path */) {}
virtual bool supportsCache() const { return false; }
@ -206,14 +203,14 @@ public:
virtual WriteSettings getAdjustedSettingsFromMetadataFile(const WriteSettings & settings, const std::string & /* path */) const { return settings; }
virtual ReadSettings patchSettings(const ReadSettings & read_settings) const;
virtual WriteSettings patchSettings(const WriteSettings & write_settings) const;
protected:
/// Should be called from implementation of applyNewSettings()
void applyRemoteThrottlingSettings(ContextPtr context);
/// Should be used by implementation of read* and write* methods
virtual ReadSettings patchSettings(const ReadSettings & read_settings) const;
virtual WriteSettings patchSettings(const WriteSettings & write_settings) const;
private:
mutable std::mutex throttlers_mutex;
ThrottlerPtr remote_read_throttler;

View File

@ -1,10 +1,17 @@
#include <Disks/ObjectStorages/LocalObjectStorage.h>
#include <Disks/ObjectStorages/Local/LocalObjectStorage.h>
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
#include <Interpreters/Context.h>
#include <Common/filesystemHelpers.h>
#include <Common/logger_useful.h>
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
#include <Disks/IO/createReadBufferFromFileBase.h>
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
#include <IO/SeekAvoidingReadBuffer.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/copyData.h>
#include <Common/getRandomASCIIString.h>
#include <filesystem>
namespace fs = std::filesystem;
@ -14,12 +21,8 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
}
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int CANNOT_UNLINK;
}
@ -38,7 +41,7 @@ LocalObjectStorage::LocalObjectStorage()
bool LocalObjectStorage::exists(const StoredObject & object) const
{
return fs::exists(object.absolute_path);
return fs::exists(object.remote_path);
}
std::unique_ptr<ReadBufferFromFileBase> LocalObjectStorage::readObjects( /// NOLINT
@ -47,30 +50,39 @@ std::unique_ptr<ReadBufferFromFileBase> LocalObjectStorage::readObjects( /// NOL
std::optional<size_t> read_hint,
std::optional<size_t> file_size) const
{
if (objects.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "LocalObjectStorage support read only from single object");
auto modified_settings = patchSettings(read_settings);
auto read_buffer_creator =
[=] (const std::string & file_path, size_t /* read_until_position */)
-> std::unique_ptr<ReadBufferFromFileBase>
{
return createReadBufferFromFileBase(file_path, modified_settings, read_hint, file_size);
};
return readObject(objects[0], read_settings, read_hint, file_size);
auto impl = std::make_unique<ReadBufferFromRemoteFSGather>(
std::move(read_buffer_creator), objects, modified_settings);
/// We use `remove_fs_method` (not `local_fs_method`) because we are about to use
/// AsynchronousReadIndirectBufferFromRemoteFS which works by the remote_fs_* settings.
if (modified_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
{
auto & reader = getThreadPoolReader();
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, modified_settings, std::move(impl));
}
else
{
auto buf = std::make_unique<ReadIndirectBufferFromRemoteFS>(std::move(impl), modified_settings);
return std::make_unique<SeekAvoidingReadBuffer>(
std::move(buf), modified_settings.remote_read_min_bytes_for_seek);
}
}
std::string LocalObjectStorage::getUniqueId(const std::string & path) const
ReadSettings LocalObjectStorage::patchSettings(const ReadSettings & read_settings) const
{
return toString(getINodeNumberFromPath(path));
}
std::unique_ptr<ReadBufferFromFileBase> LocalObjectStorage::readObject( /// NOLINT
const StoredObject & object,
const ReadSettings & read_settings,
std::optional<size_t> read_hint,
std::optional<size_t> file_size) const
{
const auto & path = object.absolute_path;
if (!file_size)
file_size = tryGetSizeFromFilePath(path);
if (!read_settings.enable_filesystem_cache)
return IObjectStorage::patchSettings(read_settings);
auto modified_settings{read_settings};
/// For now we cannot allow asynchronous reader from local filesystem when CachedObjectStorage is used.
ReadSettings modified_settings{read_settings};
switch (modified_settings.local_fs_method)
{
case LocalFSReadMethod::pread_threadpool:
@ -85,23 +97,39 @@ std::unique_ptr<ReadBufferFromFileBase> LocalObjectStorage::readObject( /// NOLI
break;
}
}
return IObjectStorage::patchSettings(modified_settings);
}
std::unique_ptr<ReadBufferFromFileBase> LocalObjectStorage::readObject( /// NOLINT
const StoredObject & object,
const ReadSettings & read_settings,
std::optional<size_t> read_hint,
std::optional<size_t> file_size) const
{
const auto & path = object.remote_path;
if (!file_size)
file_size = tryGetSizeFromFilePath(path);
LOG_TEST(log, "Read object: {}", path);
return createReadBufferFromFileBase(path, modified_settings, read_hint, file_size);
return createReadBufferFromFileBase(path, patchSettings(read_settings), read_hint, file_size);
}
std::unique_ptr<WriteBufferFromFileBase> LocalObjectStorage::writeObject( /// NOLINT
const StoredObject & object,
WriteMode mode,
std::optional<ObjectAttributes> /* attributes */,
FinalizeCallback && /* finalize_callback */,
FinalizeCallback && finalize_callback,
size_t buf_size,
const WriteSettings & /* write_settings */)
{
const auto & path = object.absolute_path;
int flags = (mode == WriteMode::Append) ? (O_APPEND | O_CREAT | O_WRONLY) : -1;
LOG_TEST(log, "Write object: {}", path);
return std::make_unique<WriteBufferFromFile>(path, buf_size, flags);
if (mode != WriteMode::Rewrite)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "LocalObjectStorage doesn't support append to files");
LOG_TEST(log, "Write object: {}", object.remote_path);
auto impl = std::make_unique<WriteBufferFromFile>(object.remote_path, buf_size);
return std::make_unique<WriteIndirectBufferFromRemoteFS>(
std::move(impl), std::move(finalize_callback), object.remote_path);
}
void LocalObjectStorage::removeObject(const StoredObject & object)
@ -110,8 +138,8 @@ void LocalObjectStorage::removeObject(const StoredObject & object)
if (!exists(object))
return;
if (0 != unlink(object.absolute_path.data()))
throwFromErrnoWithPath("Cannot unlink file " + object.absolute_path, object.absolute_path, ErrorCodes::CANNOT_UNLINK);
if (0 != unlink(object.remote_path.data()))
throwFromErrnoWithPath("Cannot unlink file " + object.remote_path, object.remote_path, ErrorCodes::CANNOT_UNLINK);
}
void LocalObjectStorage::removeObjects(const StoredObjects & objects)
@ -140,16 +168,10 @@ ObjectMetadata LocalObjectStorage::getObjectMetadata(const std::string & /* path
void LocalObjectStorage::copyObject( // NOLINT
const StoredObject & object_from, const StoredObject & object_to, std::optional<ObjectAttributes> /* object_to_attributes */)
{
fs::path to = object_to.absolute_path;
fs::path from = object_from.absolute_path;
/// Same logic as in DiskLocal.
if (object_from.absolute_path.ends_with('/'))
from = from.parent_path();
if (fs::is_directory(from))
to /= from.filename();
fs::copy(from, to, fs::copy_options::recursive | fs::copy_options::overwrite_existing);
auto in = readObject(object_from);
auto out = writeObject(object_to, WriteMode::Rewrite);
copyData(*in, *out);
out->finalize();
}
void LocalObjectStorage::shutdown()
@ -173,4 +195,10 @@ void LocalObjectStorage::applyNewSettings(
{
}
std::string LocalObjectStorage::generateBlobNameForPath(const std::string & /* path */)
{
constexpr size_t key_name_total_size = 32;
return getRandomASCIIString(key_name_total_size);
}
}

View File

@ -77,14 +77,12 @@ public:
const std::string & config_prefix,
ContextPtr context) override;
bool supportsAppend() const override { return true; }
std::string generateBlobNameForPath(const std::string & path) override { return path; }
std::string getUniqueId(const std::string & path) const override;
std::string generateBlobNameForPath(const std::string & path) override;
bool isRemote() const override { return false; }
ReadSettings patchSettings(const ReadSettings & read_settings) const override;
private:
Poco::Logger * log;
DataSourceDescription data_source_description;

View File

@ -0,0 +1,44 @@
#include <Disks/DiskFactory.h>
#include <Disks/loadLocalDiskConfig.h>
#include <Disks/ObjectStorages/Local/LocalObjectStorage.h>
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
#include <Disks/ObjectStorages/MetadataStorageFromDisk.h>
#include <Disks/ObjectStorages/DiskObjectStorage.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
void registerDiskLocalObjectStorage(DiskFactory & factory, bool global_skip_access_check)
{
auto creator = [global_skip_access_check](
const String & name,
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
ContextPtr context,
const DisksMap & /*map*/) -> DiskPtr
{
String path;
UInt64 keep_free_space_bytes;
loadDiskLocalConfig(name, config, config_prefix, context, path, keep_free_space_bytes);
fs::create_directories(path);
String type = config.getString(config_prefix + ".type");
chassert(type == "local_blob_storage");
std::shared_ptr<LocalObjectStorage> local_storage = std::make_shared<LocalObjectStorage>();
MetadataStoragePtr metadata_storage;
auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context);
metadata_storage = std::make_shared<MetadataStorageFromDisk>(metadata_disk, path);
auto disk = std::make_shared<DiskObjectStorage>(
name, path, "Local", metadata_storage, local_storage, false, /* threadpool_size */16);
disk->startup(context, global_skip_access_check);
return disk;
};
factory.registerDiskType("local_blob_storage", creator);
}
}

View File

@ -145,7 +145,7 @@ StoredObjects MetadataStorageFromDisk::getStorageObjects(const std::string & pat
for (auto & [object_relative_path, size] : object_storage_relative_paths)
{
auto object_path = fs::path(metadata->getBlobsCommonPrefix()) / object_relative_path;
StoredObject object{ object_path, size, path, [](const String & path_){ return path_; }};
StoredObject object{ object_path, size, path };
object_storage_paths.push_back(object);
}

View File

@ -108,7 +108,7 @@ StoredObjects MetadataStorageFromPlainObjectStorage::getStorageObjects(const std
{
std::string blob_name = object_storage->generateBlobNameForPath(path);
size_t object_size = getFileSize(blob_name);
auto object = StoredObject::create(*object_storage, getAbsolutePath(blob_name), object_size, path, /* exists */true);
auto object = StoredObject(getAbsolutePath(blob_name), object_size, path);
return {std::move(object)};
}
@ -119,7 +119,7 @@ const IMetadataStorage & MetadataStorageFromPlainObjectStorageTransaction::getSt
void MetadataStorageFromPlainObjectStorageTransaction::unlinkFile(const std::string & path)
{
auto object = StoredObject::create(*metadata_storage.object_storage, metadata_storage.getAbsolutePath(path));
auto object = StoredObject(metadata_storage.getAbsolutePath(path));
metadata_storage.object_storage->removeObject(object);
}

View File

@ -19,7 +19,6 @@
#include <Interpreters/threadPoolCallbackRunner.h>
#include <Disks/ObjectStorages/S3/diskSettings.h>
#include <Common/getRandomASCIIString.h>
#include <Common/ProfileEvents.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/logger_useful.h>
@ -86,27 +85,10 @@ void logIfError(const Aws::Utils::Outcome<Result, Error> & response, std::functi
}
std::string S3ObjectStorage::generateBlobNameForPath(const std::string & /* path */)
{
/// Path to store the new S3 object.
/// Total length is 32 a-z characters for enough randomness.
/// First 3 characters are used as a prefix for
/// https://aws.amazon.com/premiumsupport/knowledge-center/s3-object-key-naming-pattern/
constexpr size_t key_name_total_size = 32;
constexpr size_t key_name_prefix_size = 3;
/// Path to store new S3 object.
return fmt::format("{}/{}",
getRandomASCIIString(key_name_prefix_size),
getRandomASCIIString(key_name_total_size - key_name_prefix_size));
}
bool S3ObjectStorage::exists(const StoredObject & object) const
{
auto settings_ptr = s3_settings.get();
return S3::objectExists(*client.get(), bucket, object.absolute_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true);
return S3::objectExists(*client.get(), bucket, object.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true);
}
std::unique_ptr<ReadBufferFromFileBase> S3ObjectStorage::readObjects( /// NOLINT
@ -115,17 +97,15 @@ std::unique_ptr<ReadBufferFromFileBase> S3ObjectStorage::readObjects( /// NOLINT
std::optional<size_t>,
std::optional<size_t>) const
{
assert(!objects[0].getPathKeyForCache().empty());
ReadSettings disk_read_settings = patchSettings(read_settings);
auto settings_ptr = s3_settings.get();
auto read_buffer_creator =
[this, settings_ptr, disk_read_settings]
(const std::string & path, size_t read_until_position) -> std::shared_ptr<ReadBufferFromFileBase>
(const std::string & path, size_t read_until_position) -> std::unique_ptr<ReadBufferFromFileBase>
{
return std::make_shared<ReadBufferFromS3>(
return std::make_unique<ReadBufferFromS3>(
client.get(),
bucket,
path,
@ -146,7 +126,8 @@ std::unique_ptr<ReadBufferFromFileBase> S3ObjectStorage::readObjects( /// NOLINT
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
{
auto & reader = getThreadPoolReader();
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, disk_read_settings, std::move(s3_impl));
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(
reader, disk_read_settings, std::move(s3_impl), disk_read_settings.remote_read_min_bytes_for_seek);
}
else
{
@ -165,7 +146,7 @@ std::unique_ptr<ReadBufferFromFileBase> S3ObjectStorage::readObject( /// NOLINT
return std::make_unique<ReadBufferFromS3>(
client.get(),
bucket,
object.absolute_path,
object.remote_path,
version_id,
settings_ptr->request_settings,
patchSettings(read_settings));
@ -192,7 +173,7 @@ std::unique_ptr<WriteBufferFromFileBase> S3ObjectStorage::writeObject( /// NOLIN
auto s3_buffer = std::make_unique<WriteBufferFromS3>(
client.get(),
bucket,
object.absolute_path,
object.remote_path,
settings_ptr->request_settings,
attributes,
buf_size,
@ -200,7 +181,7 @@ std::unique_ptr<WriteBufferFromFileBase> S3ObjectStorage::writeObject( /// NOLIN
disk_write_settings);
return std::make_unique<WriteIndirectBufferFromRemoteFS>(
std::move(s3_buffer), std::move(finalize_callback), object.absolute_path);
std::move(s3_buffer), std::move(finalize_callback), object.remote_path);
}
void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const
@ -301,12 +282,12 @@ void S3ObjectStorage::removeObjectImpl(const StoredObject & object, bool if_exis
ProfileEvents::increment(ProfileEvents::DiskS3DeleteObjects);
S3::DeleteObjectRequest request;
request.SetBucket(bucket);
request.SetKey(object.absolute_path);
request.SetKey(object.remote_path);
auto outcome = client_ptr->DeleteObject(request);
throwIfUnexpectedError(outcome, if_exists);
LOG_TRACE(log, "Object with path {} was removed from S3", object.absolute_path);
LOG_TRACE(log, "Object with path {} was removed from S3", object.remote_path);
}
void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_exists)
@ -334,12 +315,12 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e
for (; current_position < objects.size() && current_chunk.size() < chunk_size_limit; ++current_position)
{
Aws::S3::Model::ObjectIdentifier obj;
obj.SetKey(objects[current_position].absolute_path);
obj.SetKey(objects[current_position].remote_path);
current_chunk.push_back(obj);
if (!keys.empty())
keys += ", ";
keys += objects[current_position].absolute_path;
keys += objects[current_position].remote_path;
}
Aws::S3::Model::Delete delkeys;
@ -403,9 +384,9 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT
{
auto client_ptr = client.get();
auto settings_ptr = s3_settings.get();
auto size = S3::getObjectSize(*client_ptr, bucket, object_from.absolute_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true);
auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true);
auto scheduler = threadPoolCallbackRunner<void>(getThreadPoolWriter(), "S3ObjStor_copy");
copyS3File(client_ptr, bucket, object_from.absolute_path, 0, size, dest_s3->bucket, object_to.absolute_path,
copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path,
settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true);
}
else
@ -419,9 +400,9 @@ void S3ObjectStorage::copyObject( // NOLINT
{
auto client_ptr = client.get();
auto settings_ptr = s3_settings.get();
auto size = S3::getObjectSize(*client_ptr, bucket, object_from.absolute_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true);
auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true);
auto scheduler = threadPoolCallbackRunner<void>(getThreadPoolWriter(), "S3ObjStor_copy");
copyS3File(client_ptr, bucket, object_from.absolute_path, 0, size, bucket, object_to.absolute_path,
copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path,
settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true);
}

View File

@ -144,8 +144,6 @@ public:
std::string getObjectsNamespace() const override { return bucket; }
std::string generateBlobNameForPath(const std::string & path) override;
bool isRemote() const override { return true; }
void setCapabilitiesSupportBatchDelete(bool value) { s3_capabilities.support_batch_delete = value; }

View File

@ -55,7 +55,7 @@ void copyS3FileToDisk(
/// Object storage always uses mode `Rewrite` because it simulates append using metadata and different files.
chassert(write_mode_ == WriteMode::Rewrite);
copyS3File(s3_client, src_bucket, src_key, *src_offset, *src_size, dest_bucket, /* dest_key= */ object_.absolute_path,
copyS3File(s3_client, src_bucket, src_key, *src_offset, *src_size, dest_bucket, /* dest_key= */ object_.remote_path,
request_settings, object_attributes_, scheduler, /* for_disk_s3= */ true);
return *src_size;

View File

@ -142,6 +142,9 @@ std::unique_ptr<S3::Client> getClient(
= [proxy_config](const auto & request_config) { proxy_config->errorReport(request_config); };
}
HTTPHeaderEntries headers = S3::getHTTPHeaders(config_prefix, config);
S3::ServerSideEncryptionKMSConfig sse_kms_config = S3::getSSEKMSConfig(config_prefix, config);
client_configuration.retryStrategy
= std::make_shared<Aws::Client::DefaultRetryStrategy>(config.getUInt(config_prefix + ".retry_attempts", 10));
@ -151,7 +154,8 @@ std::unique_ptr<S3::Client> getClient(
config.getString(config_prefix + ".access_key_id", ""),
config.getString(config_prefix + ".secret_access_key", ""),
config.getString(config_prefix + ".server_side_encryption_customer_key_base64", ""),
{},
std::move(sse_kms_config),
std::move(headers),
S3::CredentialsConfiguration
{
config.getBool(config_prefix + ".use_environment_credentials", config.getBool("s3.use_environment_credentials", true)),

View File

@ -174,6 +174,6 @@ void registerDiskS3(DiskFactory & factory, bool global_skip_access_check)
#else
void registerDiskS3(DiskFactory &, bool /* global_skip_access_check */) {}
void registerDiskS3(DB::DiskFactory &, bool /* global_skip_access_check */) {}
#endif

View File

@ -1,72 +0,0 @@
#include <Disks/ObjectStorages/StoredObject.h>
#include <Disks/ObjectStorages/IMetadataStorage.h>
#include <Disks/ObjectStorages/IObjectStorage.h>
#include <Common/logger_useful.h>
namespace DB
{
StoredObject::StoredObject(
const std::string & absolute_path_,
uint64_t bytes_size_,
const std::string & mapped_path_,
PathKeyForCacheCreator && path_key_for_cache_creator_)
: absolute_path(absolute_path_)
, mapped_path(mapped_path_)
, bytes_size(bytes_size_)
, path_key_for_cache_creator(std::move(path_key_for_cache_creator_))
{
}
std::string StoredObject::getPathKeyForCache() const
{
if (!path_key_for_cache_creator)
return ""; /// This empty result need to be used with care.
return path_key_for_cache_creator(absolute_path);
}
const std::string & StoredObject::getMappedPath() const
{
return mapped_path;
}
StoredObject StoredObject::create(
const IObjectStorage & object_storage,
const std::string & object_path,
size_t object_size,
const std::string & mapped_path,
bool exists,
bool object_bypasses_cache)
{
if (object_bypasses_cache)
return StoredObject(object_path, object_size, mapped_path, {});
PathKeyForCacheCreator path_key_for_cache_creator = [&object_storage](const std::string & path) -> std::string
{
try
{
return object_storage.getUniqueId(path);
}
catch (...)
{
LOG_DEBUG(
&Poco::Logger::get("StoredObject"),
"Object does not exist while getting cache path hint (object path: {})",
path);
return "";
}
};
if (exists)
{
path_key_for_cache_creator = [path = path_key_for_cache_creator(object_path)](const std::string &) { return path; };
}
return StoredObject(object_path, object_size, mapped_path, std::move(path_key_for_cache_creator));
}
}

View File

@ -11,38 +11,20 @@ namespace DB
/// Object metadata: path, size, path_key_for_cache.
struct StoredObject
{
/// Absolute path of the blob in object storage.
std::string absolute_path;
/// A map which is mapped to current blob (for example, a corresponding local path as clickhouse sees it).
std::string mapped_path;
std::string remote_path;
std::string local_path; /// or equivalent "metadata_path"
uint64_t bytes_size = 0;
std::string getPathKeyForCache() const;
const std::string & getMappedPath() const;
/// Create `StoredObject` based on metadata storage and blob name of the object.
static StoredObject create(
const IObjectStorage & object_storage,
const std::string & object_path,
size_t object_size = 0,
const std::string & mapped_path = "",
bool exists = false,
bool object_bypasses_cache = false);
/// Optional hint for cache. Use delayed initialization
/// because somecache hint implementation requires it.
using PathKeyForCacheCreator = std::function<std::string(const std::string &)>;
PathKeyForCacheCreator path_key_for_cache_creator;
StoredObject() = default;
explicit StoredObject(
const std::string & absolute_path_,
const std::string & remote_path_,
uint64_t bytes_size_ = 0,
const std::string & mapped_path_ = "",
PathKeyForCacheCreator && path_key_for_cache_creator_ = {});
const std::string & local_path_ = "")
: remote_path(remote_path_)
, local_path(local_path_)
, bytes_size(bytes_size_) {}
};
using StoredObjects = std::vector<StoredObject>;

View File

@ -108,7 +108,7 @@ StoredObjects MetadataStorageFromStaticFilesWebServer::getStorageObjects(const s
auto fs_path = fs::path(object_storage.url) / path;
std::string remote_path = fs_path.parent_path() / (escapeForFileName(fs_path.stem()) + fs_path.extension().string());
remote_path = remote_path.substr(object_storage.url.size());
return {StoredObject::create(object_storage, remote_path, object_storage.files.at(path).size, path, true)};
return {StoredObject(remote_path, object_storage.files.at(path).size, path)};
}
std::vector<std::string> MetadataStorageFromStaticFilesWebServer::listDirectory(const std::string & path) const

View File

@ -119,7 +119,7 @@ WebObjectStorage::WebObjectStorage(
bool WebObjectStorage::exists(const StoredObject & object) const
{
const auto & path = object.absolute_path;
const auto & path = object.remote_path;
LOG_TRACE(&Poco::Logger::get("DiskWeb"), "Checking existence of path: {}", path);
@ -169,9 +169,9 @@ std::unique_ptr<ReadBufferFromFileBase> WebObjectStorage::readObject( /// NOLINT
{
auto read_buffer_creator =
[this, read_settings]
(const std::string & path_, size_t read_until_position) -> std::shared_ptr<ReadBufferFromFileBase>
(const std::string & path_, size_t read_until_position) -> std::unique_ptr<ReadBufferFromFileBase>
{
return std::make_shared<ReadBufferFromWebServer>(
return std::make_unique<ReadBufferFromWebServer>(
fs::path(url) / path_,
getContext(),
read_settings,

View File

@ -87,8 +87,6 @@ public:
const std::string & config_prefix,
ContextPtr context) override;
bool supportsAppend() const override { return false; }
std::string generateBlobNameForPath(const std::string & path) override { return path; }
bool isRemote() const override { return true; }

View File

@ -0,0 +1,63 @@
#include <Disks/loadLocalDiskConfig.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Interpreters/Context.h>
#include <Disks/DiskLocal.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
}
void loadDiskLocalConfig(const String & name,
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
ContextPtr context,
String & path,
UInt64 & keep_free_space_bytes)
{
path = config.getString(config_prefix + ".path", "");
if (name == "default")
{
if (!path.empty())
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG,
"\"default\" disk path should be provided in <path> not it <storage_configuration>");
path = context->getPath();
}
else
{
if (path.empty())
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Disk path can not be empty. Disk {}", name);
if (path.back() != '/')
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Disk path must end with /. Disk {}", name);
if (path == context->getPath())
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Disk path ('{}') cannot be equal to <path>. Use <default> disk instead.", path);
}
bool has_space_ratio = config.has(config_prefix + ".keep_free_space_ratio");
if (config.has(config_prefix + ".keep_free_space_bytes") && has_space_ratio)
throw Exception(ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG,
"Only one of 'keep_free_space_bytes' and 'keep_free_space_ratio' can be specified");
keep_free_space_bytes = config.getUInt64(config_prefix + ".keep_free_space_bytes", 0);
if (has_space_ratio)
{
auto ratio = config.getDouble(config_prefix + ".keep_free_space_ratio");
if (ratio < 0 || ratio > 1)
throw Exception(ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG, "'keep_free_space_ratio' have to be between 0 and 1");
String tmp_path = path;
if (tmp_path.empty())
tmp_path = context->getPath();
// Create tmp disk for getting total disk space.
keep_free_space_bytes = static_cast<UInt64>(DiskLocal("tmp", tmp_path, 0).getTotalSpace() * ratio);
}
}
}

View File

@ -0,0 +1,16 @@
#pragma once
#include <Core/Types.h>
#include <Interpreters/Context_fwd.h>
namespace Poco::Util { class AbstractConfiguration; }
namespace DB
{
void loadDiskLocalConfig(
const String & name,
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
ContextPtr context,
String & path,
UInt64 & keep_free_space_bytes);
}

View File

@ -29,6 +29,9 @@ void registerDiskWebServer(DiskFactory & factory, bool global_skip_access_check)
void registerDiskCache(DiskFactory & factory, bool global_skip_access_check);
void registerDiskLocalObjectStorage(DiskFactory & factory, bool global_skip_access_check);
void registerDisks(bool global_skip_access_check)
{
auto & factory = DiskFactory::instance();
@ -54,6 +57,8 @@ void registerDisks(bool global_skip_access_check)
registerDiskWebServer(factory, global_skip_access_check);
registerDiskCache(factory, global_skip_access_check);
registerDiskLocalObjectStorage(factory, global_skip_access_check);
}
}

View File

@ -110,7 +110,8 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.json.allow_object_type = context->getSettingsRef().allow_experimental_object_type;
format_settings.null_as_default = settings.input_format_null_as_default;
format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros;
format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size;
format_settings.parquet.row_group_rows = settings.output_format_parquet_row_group_size;
format_settings.parquet.row_group_bytes = settings.output_format_parquet_row_group_size_bytes;
format_settings.parquet.output_version = settings.output_format_parquet_version;
format_settings.parquet.import_nested = settings.input_format_parquet_import_nested;
format_settings.parquet.case_insensitive_column_matching = settings.input_format_parquet_case_insensitive_column_matching;
@ -734,6 +735,14 @@ void FormatFactory::markFormatSupportsSubcolumns(const String & name)
target = true;
}
void FormatFactory::markOutputFormatPrefersLargeBlocks(const String & name)
{
auto & target = dict[name].prefers_large_blocks;
if (target)
throw Exception(ErrorCodes::LOGICAL_ERROR, "FormatFactory: Format {} is already marked as preferring large blocks", name);
target = true;
}
bool FormatFactory::checkIfFormatSupportsSubcolumns(const String & name) const
{
const auto & target = getCreators(name);
@ -794,6 +803,12 @@ bool FormatFactory::checkIfFormatHasAnySchemaReader(const String & name) const
return checkIfFormatHasSchemaReader(name) || checkIfFormatHasExternalSchemaReader(name);
}
bool FormatFactory::checkIfOutputFormatPrefersLargeBlocks(const String & name) const
{
const auto & target = getCreators(name);
return target.prefers_large_blocks;
}
void FormatFactory::checkFormatName(const String & name) const
{
auto it = dict.find(name);

View File

@ -141,6 +141,7 @@ private:
bool supports_parallel_formatting{false};
bool supports_subcolumns{false};
bool supports_subset_of_columns{false};
bool prefers_large_blocks{false};
NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker;
AppendSupportChecker append_support_checker;
AdditionalInfoForSchemaCacheGetter additional_info_for_schema_cache_getter;
@ -237,6 +238,7 @@ public:
void registerExternalSchemaReader(const String & name, ExternalSchemaReaderCreator external_schema_reader_creator);
void markOutputFormatSupportsParallelFormatting(const String & name);
void markOutputFormatPrefersLargeBlocks(const String & name);
void markFormatSupportsSubcolumns(const String & name);
void markFormatSupportsSubsetOfColumns(const String & name);
@ -246,6 +248,7 @@ public:
bool checkIfFormatHasSchemaReader(const String & name) const;
bool checkIfFormatHasExternalSchemaReader(const String & name) const;
bool checkIfFormatHasAnySchemaReader(const String & name) const;
bool checkIfOutputFormatPrefersLargeBlocks(const String & name) const;
void registerAdditionalInfoForSchemaCacheGetter(const String & name, AdditionalInfoForSchemaCacheGetter additional_info_for_schema_cache_getter);
String getAdditionalInfoForSchemaCache(const String & name, ContextPtr context, const std::optional<FormatSettings> & format_settings_ = std::nullopt);

View File

@ -205,7 +205,8 @@ struct FormatSettings
struct
{
UInt64 row_group_size = 1000000;
UInt64 row_group_rows = 1000000;
UInt64 row_group_bytes = 512 * 1024 * 1024;
bool import_nested = false;
bool allow_missing_columns = false;
bool skip_columns_with_unsupported_types_in_schema_inference = false;

View File

@ -98,10 +98,6 @@ struct ReadSettings
bool read_from_filesystem_cache_if_exists_otherwise_bypass_cache = false;
bool enable_filesystem_cache_log = false;
bool is_file_cache_persistent = false; /// Some files can be made non-evictable.
/// Some buffers which read via thread pool can also do caching in threadpool
/// (instead of caching the result outside of threadpool). By default, if they support it,
/// they will do it. But this behaviour can be changed with this setting.
bool enable_filesystem_cache_on_lower_level = true;
size_t filesystem_cache_max_download_size = (128UL * 1024 * 1024 * 1024);
bool skip_download_if_exceeds_query_cache = true;

View File

@ -96,6 +96,7 @@ void verifyClientConfiguration(const Aws::Client::ClientConfiguration & client_c
std::unique_ptr<Client> Client::create(
size_t max_redirects_,
ServerSideEncryptionKMSConfig sse_kms_config_,
const std::shared_ptr<Aws::Auth::AWSCredentialsProvider> & credentials_provider,
const Aws::Client::ClientConfiguration & client_configuration,
Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads,
@ -103,7 +104,7 @@ std::unique_ptr<Client> Client::create(
{
verifyClientConfiguration(client_configuration);
return std::unique_ptr<Client>(
new Client(max_redirects_, credentials_provider, client_configuration, sign_payloads, use_virtual_addressing));
new Client(max_redirects_, std::move(sse_kms_config_), credentials_provider, client_configuration, sign_payloads, use_virtual_addressing));
}
std::unique_ptr<Client> Client::create(const Client & other)
@ -113,12 +114,14 @@ std::unique_ptr<Client> Client::create(const Client & other)
Client::Client(
size_t max_redirects_,
ServerSideEncryptionKMSConfig sse_kms_config_,
const std::shared_ptr<Aws::Auth::AWSCredentialsProvider> & credentials_provider,
const Aws::Client::ClientConfiguration & client_configuration,
Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads,
bool use_virtual_addressing)
: Aws::S3::S3Client(credentials_provider, client_configuration, std::move(sign_payloads), use_virtual_addressing)
, max_redirects(max_redirects_)
, sse_kms_config(std::move(sse_kms_config_))
, log(&Poco::Logger::get("S3Client"))
{
auto * endpoint_provider = dynamic_cast<Aws::S3::Endpoint::S3DefaultEpProviderBase *>(accessEndpointProvider().get());
@ -141,6 +144,7 @@ Client::Client(const Client & other)
, detect_region(other.detect_region)
, provider_type(other.provider_type)
, max_redirects(other.max_redirects)
, sse_kms_config(other.sse_kms_config)
, log(&Poco::Logger::get("S3Client"))
{
cache = std::make_shared<ClientCache>(*other.cache);
@ -178,6 +182,28 @@ void Client::insertRegionOverride(const std::string & bucket, const std::string
LOG_INFO(log, "Detected different region ('{}') for bucket {} than the one defined ('{}')", region, bucket, explicit_region);
}
template <typename RequestType>
void Client::setKMSHeaders(RequestType & request) const
{
// Don't do anything unless a key ID was specified
if (sse_kms_config.key_id)
{
request.SetServerSideEncryption(Model::ServerSideEncryption::aws_kms);
// If the key ID was specified but is empty, treat it as using the AWS managed key and omit the header
if (!sse_kms_config.key_id->empty())
request.SetSSEKMSKeyId(*sse_kms_config.key_id);
if (sse_kms_config.encryption_context)
request.SetSSEKMSEncryptionContext(*sse_kms_config.encryption_context);
if (sse_kms_config.bucket_key_enabled)
request.SetBucketKeyEnabled(*sse_kms_config.bucket_key_enabled);
}
}
// Explicitly instantiate this method only for the request types that support KMS headers
template void Client::setKMSHeaders<CreateMultipartUploadRequest>(CreateMultipartUploadRequest & request) const;
template void Client::setKMSHeaders<CopyObjectRequest>(CopyObjectRequest & request) const;
template void Client::setKMSHeaders<PutObjectRequest>(PutObjectRequest & request) const;
Model::HeadObjectOutcome Client::HeadObject(const HeadObjectRequest & request) const
{
const auto & bucket = request.GetBucket();
@ -574,6 +600,7 @@ std::unique_ptr<S3::Client> ClientFactory::create( // NOLINT
const String & access_key_id,
const String & secret_access_key,
const String & server_side_encryption_customer_key_base64,
ServerSideEncryptionKMSConfig sse_kms_config,
HTTPHeaderEntries headers,
CredentialsConfiguration credentials_configuration)
{
@ -596,6 +623,7 @@ std::unique_ptr<S3::Client> ClientFactory::create( // NOLINT
Aws::Utils::HashingUtils::Base64Encode(Aws::Utils::HashingUtils::CalculateMD5(str_buffer))});
}
// These will be added after request signing
client_configuration.extra_headers = std::move(headers);
Aws::Auth::AWSCredentials credentials(access_key_id, secret_access_key);
@ -607,6 +635,7 @@ std::unique_ptr<S3::Client> ClientFactory::create( // NOLINT
client_configuration.retryStrategy = std::make_shared<Client::RetryStrategy>(std::move(client_configuration.retryStrategy));
return Client::create(
client_configuration.s3_max_redirects,
std::move(sse_kms_config),
credentials_provider,
client_configuration, // Client configuration.
Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never,

View File

@ -1,7 +1,27 @@
#pragma once
#include <optional>
#include <base/types.h>
#include "config.h"
namespace DB::S3
{
/// See https://docs.aws.amazon.com/AmazonS3/latest/userguide/specifying-kms-encryption.html
/// Needed by S3Common.h even if USE_AWS_S3 is 0
struct ServerSideEncryptionKMSConfig
{
// If key_id is non-null, enable SSE-KMS. If key_id is "", use the AWS managed key
std::optional<String> key_id = std::nullopt;
std::optional<String> encryption_context = std::nullopt;
std::optional<bool> bucket_key_enabled = std::nullopt;
bool operator==(const ServerSideEncryptionKMSConfig & other) const = default;
};
}
#if USE_AWS_S3
#include <Common/assert_cast.h>
@ -85,6 +105,7 @@ public:
/// e.g. Client::RetryStrategy should be used
static std::unique_ptr<Client> create(
size_t max_redirects_,
ServerSideEncryptionKMSConfig sse_kms_config_,
const std::shared_ptr<Aws::Auth::AWSCredentialsProvider> & credentials_provider,
const Aws::Client::ClientConfiguration & client_configuration,
Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads,
@ -143,6 +164,13 @@ public:
std::shared_ptr<Aws::Client::RetryStrategy> wrapped_strategy;
};
/// SSE-KMS headers MUST be signed, so they need to be added before the SDK signs the message
/// (before sending the request with one of the methods below).
/// Per the docs (https://docs.aws.amazon.com/AmazonS3/latest/userguide/specifying-kms-encryption.html),
/// the headers should only be set for PutObject, CopyObject, POST Object, and CreateMultipartUpload.
template <typename RequestType>
void setKMSHeaders(RequestType & request) const;
Model::HeadObjectOutcome HeadObject(const HeadObjectRequest & request) const;
Model::ListObjectsV2Outcome ListObjectsV2(const ListObjectsV2Request & request) const;
Model::ListObjectsOutcome ListObjects(const ListObjectsRequest & request) const;
@ -165,6 +193,7 @@ public:
ProviderType getProviderType() const;
private:
Client(size_t max_redirects_,
ServerSideEncryptionKMSConfig sse_kms_config_,
const std::shared_ptr<Aws::Auth::AWSCredentialsProvider>& credentials_provider,
const Aws::Client::ClientConfiguration& client_configuration,
Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads,
@ -215,6 +244,8 @@ private:
const size_t max_redirects;
const ServerSideEncryptionKMSConfig sse_kms_config;
Poco::Logger * log;
};
@ -231,6 +262,7 @@ public:
const String & access_key_id,
const String & secret_access_key,
const String & server_side_encryption_customer_key_base64,
ServerSideEncryptionKMSConfig sse_kms_config,
HTTPHeaderEntries headers,
CredentialsConfiguration credentials_configuration);

View File

@ -100,9 +100,8 @@ namespace
std::mutex bg_tasks_mutex;
std::condition_variable bg_tasks_condvar;
void createMultipartUpload()
void fillCreateMultipartRequest(S3::CreateMultipartUploadRequest & request)
{
S3::CreateMultipartUploadRequest request;
request.SetBucket(dest_bucket);
request.SetKey(dest_key);
@ -116,6 +115,14 @@ namespace
if (!storage_class_name.empty())
request.SetStorageClass(Aws::S3::Model::StorageClassMapper::GetStorageClassForName(storage_class_name));
client_ptr->setKMSHeaders(request);
}
void createMultipartUpload()
{
S3::CreateMultipartUploadRequest request;
fillCreateMultipartRequest(request);
ProfileEvents::increment(ProfileEvents::S3CreateMultipartUpload);
if (for_disk_s3)
ProfileEvents::increment(ProfileEvents::DiskS3CreateMultipartUpload);
@ -465,6 +472,8 @@ namespace
/// If we don't do it, AWS SDK can mistakenly set it to application/xml, see https://github.com/aws/aws-sdk-cpp/issues/1840
request.SetContentType("binary/octet-stream");
client_ptr->setKMSHeaders(request);
}
void processPutRequest(const S3::PutObjectRequest & request)
@ -660,6 +669,8 @@ namespace
/// If we don't do it, AWS SDK can mistakenly set it to application/xml, see https://github.com/aws/aws-sdk-cpp/issues/1840
request.SetContentType("binary/octet-stream");
client_ptr->setKMSHeaders(request);
}
void processCopyRequest(const S3::CopyObjectRequest & request)

View File

@ -8,38 +8,66 @@
#include <Poco/Net/HTTPRequestHandlerFactory.h>
#include <Poco/Net/HTTPServer.h>
#include <Poco/Net/HTTPServerParams.h>
#include <Poco/Net/HTTPServerRequest.h>
#include <Poco/Net/HTTPServerResponse.h>
#include <Poco/Net/MessageHeader.h>
#include <Poco/Net/NetException.h>
#include <Poco/Net/ServerSocket.h>
#include <Poco/AutoPtr.h>
#include <Poco/SharedPtr.h>
class MockRequestHandler : public Poco::Net::HTTPRequestHandler
{
Poco::Net::MessageHeader & last_request_header;
public:
MockRequestHandler(Poco::Net::MessageHeader & last_request_header_)
: Poco::Net::HTTPRequestHandler(), last_request_header(last_request_header_)
{
}
void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override
{
response.setStatus(Poco::Net::HTTPResponse::HTTP_OK);
last_request_header = request;
response.send();
}
};
template <typename RequestHandler>
class HTTPRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory
{
Poco::Net::MessageHeader & last_request_header;
virtual Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest &) override
{
return new RequestHandler();
return new MockRequestHandler(last_request_header);
}
public:
HTTPRequestHandlerFactory(Poco::Net::MessageHeader & last_request_header_)
: Poco::Net::HTTPRequestHandlerFactory(), last_request_header(last_request_header_)
{
}
virtual ~HTTPRequestHandlerFactory() override
{
}
};
template <typename RequestHandler>
class TestPocoHTTPServer
{
std::unique_ptr<Poco::Net::ServerSocket> server_socket;
Poco::SharedPtr<HTTPRequestHandlerFactory<RequestHandler>> handler_factory;
Poco::SharedPtr<HTTPRequestHandlerFactory> handler_factory;
Poco::AutoPtr<Poco::Net::HTTPServerParams> server_params;
std::unique_ptr<Poco::Net::HTTPServer> server;
// Stores the last request header handled. It's obviously not thread-safe to share the same
// reference across request handlers, but it's good enough for this the purposes of this test.
Poco::Net::MessageHeader last_request_header;
public:
TestPocoHTTPServer():
server_socket(std::make_unique<Poco::Net::ServerSocket>(0)),
handler_factory(new HTTPRequestHandlerFactory<RequestHandler>()),
handler_factory(new HTTPRequestHandlerFactory(last_request_header)),
server_params(new Poco::Net::HTTPServerParams()),
server(std::make_unique<Poco::Net::HTTPServer>(handler_factory, *server_socket, server_params))
{
@ -50,4 +78,9 @@ public:
{
return "http://" + server_socket->address().toString();
}
const Poco::Net::MessageHeader & getLastRequestHeader() const
{
return last_request_header;
}
};

View File

@ -7,13 +7,9 @@
#if USE_AWS_S3
#include <memory>
#include <ostream>
#include <boost/algorithm/string.hpp>
#include <Poco/Net/HTTPRequestHandler.h>
#include <Poco/Net/HTTPServerRequest.h>
#include <Poco/Net/HTTPServerResponse.h>
#include <Poco/URI.h>
#include <aws/core/client/AWSError.h>
@ -25,6 +21,7 @@
#include <IO/ReadBufferFromS3.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadSettings.h>
#include <IO/WriteBufferFromS3.h>
#include <IO/S3Common.h>
#include <IO/S3/Client.h>
#include <IO/HTTPHeaderEntries.h>
@ -41,83 +38,33 @@ public:
~NoRetryStrategy() override = default;
};
TEST(IOTestAwsS3Client, AppendExtraSSECHeaders)
String getSSEAndSignedHeaders(const Poco::Net::MessageHeader & message_header)
{
/// See https://github.com/ClickHouse/ClickHouse/pull/19748
class MyRequestHandler : public Poco::Net::HTTPRequestHandler
String content;
for (const auto & [header_name, header_value] : message_header)
{
public:
void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override
if (boost::algorithm::starts_with(header_name, "x-amz-server-side-encryption"))
{
response.setStatus(Poco::Net::HTTPResponse::HTTP_OK);
std::ostream & out = response.send();
for (const auto & [header_name, header_value] : request)
{
if (boost::algorithm::starts_with(header_name, "x-amz-server-side-encryption-customer-"))
{
out << header_name << ": " << header_value << "\n";
}
else if (header_name == "authorization")
{
std::vector<String> parts;
boost::split(parts, header_value, [](char c){ return c == ' '; });
for (const auto & part : parts)
{
if (boost::algorithm::starts_with(part, "SignedHeaders="))
out << header_name << ": ... " << part << " ...\n";
}
}
}
out.flush();
content += header_name + ": " + header_value + "\n";
}
};
else if (header_name == "authorization")
{
std::vector<String> parts;
boost::split(parts, header_value, [](char c){ return c == ' '; });
for (const auto & part : parts)
{
if (boost::algorithm::starts_with(part, "SignedHeaders="))
content += header_name + ": ... " + part + " ...\n";
}
}
}
return content;
}
TestPocoHTTPServer<MyRequestHandler> http;
DB::RemoteHostFilter remote_host_filter;
unsigned int s3_max_redirects = 100;
DB::S3::URI uri(http.getUrl() + "/IOTestAwsS3ClientAppendExtraHeaders/test.txt");
String access_key_id = "ACCESS_KEY_ID";
String secret_access_key = "SECRET_ACCESS_KEY";
String region = "us-east-1";
void doReadRequest(std::shared_ptr<const DB::S3::Client> client, const DB::S3::URI & uri)
{
String version_id;
UInt64 max_single_read_retries = 1;
bool enable_s3_requests_logging = false;
DB::S3::PocoHTTPClientConfiguration client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(
region,
remote_host_filter,
s3_max_redirects,
enable_s3_requests_logging,
/* for_disk_s3 = */ false,
/* get_request_throttler = */ {},
/* put_request_throttler = */ {}
);
client_configuration.endpointOverride = uri.endpoint;
client_configuration.retryStrategy = std::make_shared<NoRetryStrategy>();
String server_side_encryption_customer_key_base64 = "Kv/gDqdWVGIT4iDqg+btQvV3lc1idlm4WI+MMOyHOAw=";
DB::HTTPHeaderEntries headers;
bool use_environment_credentials = false;
bool use_insecure_imds_request = false;
std::shared_ptr<DB::S3::Client> client = DB::S3::ClientFactory::instance().create(
client_configuration,
uri.is_virtual_hosted_style,
access_key_id,
secret_access_key,
server_side_encryption_customer_key_base64,
headers,
DB::S3::CredentialsConfiguration
{
.use_environment_credentials = use_environment_credentials,
.use_insecure_imds_request = use_insecure_imds_request
}
);
ASSERT_TRUE(client);
DB::ReadSettings read_settings;
DB::S3Settings::RequestSettings request_settings;
@ -133,7 +80,170 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeaders)
String content;
DB::readStringUntilEOF(content, read_buffer);
EXPECT_EQ(content, "authorization: ... SignedHeaders=amz-sdk-invocation-id;amz-sdk-request;content-type;host;x-amz-api-version;x-amz-content-sha256;x-amz-date, ...\nx-amz-server-side-encryption-customer-algorithm: AES256\nx-amz-server-side-encryption-customer-key: Kv/gDqdWVGIT4iDqg+btQvV3lc1idlm4WI+MMOyHOAw=\nx-amz-server-side-encryption-customer-key-md5: fMNuOw6OLU5GG2vc6RTA+g==\n");
}
void doWriteRequest(std::shared_ptr<const DB::S3::Client> client, const DB::S3::URI & uri)
{
UInt64 max_unexpected_write_error_retries = 1;
DB::S3Settings::RequestSettings request_settings;
request_settings.max_unexpected_write_error_retries = max_unexpected_write_error_retries;
DB::WriteBufferFromS3 write_buffer(
client,
uri.bucket,
uri.key,
request_settings
);
write_buffer.write('\0'); // doesn't matter what we write here, just needs to be something
write_buffer.finalize();
}
using RequestFn = std::function<void(std::shared_ptr<const DB::S3::Client>, const DB::S3::URI &)>;
void testServerSideEncryption(
RequestFn do_request,
String server_side_encryption_customer_key_base64,
DB::S3::ServerSideEncryptionKMSConfig sse_kms_config,
String expected_headers)
{
TestPocoHTTPServer http;
DB::RemoteHostFilter remote_host_filter;
unsigned int s3_max_redirects = 100;
DB::S3::URI uri(http.getUrl() + "/IOTestAwsS3ClientAppendExtraHeaders/test.txt");
String access_key_id = "ACCESS_KEY_ID";
String secret_access_key = "SECRET_ACCESS_KEY";
String region = "us-east-1";
bool enable_s3_requests_logging = false;
DB::S3::PocoHTTPClientConfiguration client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(
region,
remote_host_filter,
s3_max_redirects,
enable_s3_requests_logging,
/* for_disk_s3 = */ false,
/* get_request_throttler = */ {},
/* put_request_throttler = */ {}
);
client_configuration.endpointOverride = uri.endpoint;
client_configuration.retryStrategy = std::make_shared<NoRetryStrategy>();
DB::HTTPHeaderEntries headers;
bool use_environment_credentials = false;
bool use_insecure_imds_request = false;
std::shared_ptr<DB::S3::Client> client = DB::S3::ClientFactory::instance().create(
client_configuration,
uri.is_virtual_hosted_style,
access_key_id,
secret_access_key,
server_side_encryption_customer_key_base64,
sse_kms_config,
headers,
DB::S3::CredentialsConfiguration
{
.use_environment_credentials = use_environment_credentials,
.use_insecure_imds_request = use_insecure_imds_request
}
);
ASSERT_TRUE(client);
do_request(client, uri);
String content = getSSEAndSignedHeaders(http.getLastRequestHeader());
EXPECT_EQ(content, expected_headers);
}
TEST(IOTestAwsS3Client, AppendExtraSSECHeadersRead)
{
/// See https://github.com/ClickHouse/ClickHouse/pull/19748
testServerSideEncryption(
doReadRequest,
"Kv/gDqdWVGIT4iDqg+btQvV3lc1idlm4WI+MMOyHOAw=",
{},
"authorization: ... SignedHeaders="
"amz-sdk-invocation-id;"
"amz-sdk-request;"
"content-type;"
"host;"
"x-amz-api-version;"
"x-amz-content-sha256;"
"x-amz-date, ...\n"
"x-amz-server-side-encryption-customer-algorithm: AES256\n"
"x-amz-server-side-encryption-customer-key: Kv/gDqdWVGIT4iDqg+btQvV3lc1idlm4WI+MMOyHOAw=\n"
"x-amz-server-side-encryption-customer-key-md5: fMNuOw6OLU5GG2vc6RTA+g==\n");
}
TEST(IOTestAwsS3Client, AppendExtraSSECHeadersWrite)
{
/// See https://github.com/ClickHouse/ClickHouse/pull/19748
testServerSideEncryption(
doWriteRequest,
"Kv/gDqdWVGIT4iDqg+btQvV3lc1idlm4WI+MMOyHOAw=",
{},
"authorization: ... SignedHeaders="
"amz-sdk-invocation-id;"
"amz-sdk-request;"
"content-length;"
"content-md5;"
"content-type;"
"host;"
"x-amz-content-sha256;"
"x-amz-date, ...\n"
"x-amz-server-side-encryption-customer-algorithm: AES256\n"
"x-amz-server-side-encryption-customer-key: Kv/gDqdWVGIT4iDqg+btQvV3lc1idlm4WI+MMOyHOAw=\n"
"x-amz-server-side-encryption-customer-key-md5: fMNuOw6OLU5GG2vc6RTA+g==\n");
}
TEST(IOTestAwsS3Client, AppendExtraSSEKMSHeadersRead)
{
DB::S3::ServerSideEncryptionKMSConfig sse_kms_config;
sse_kms_config.key_id = "alias/test-key";
sse_kms_config.encryption_context = "arn:aws:s3:::bucket_ARN";
sse_kms_config.bucket_key_enabled = true;
// KMS headers shouldn't be set on a read request
testServerSideEncryption(
doReadRequest,
"",
sse_kms_config,
"authorization: ... SignedHeaders="
"amz-sdk-invocation-id;"
"amz-sdk-request;"
"content-type;"
"host;"
"x-amz-api-version;"
"x-amz-content-sha256;"
"x-amz-date, ...\n");
}
TEST(IOTestAwsS3Client, AppendExtraSSEKMSHeadersWrite)
{
DB::S3::ServerSideEncryptionKMSConfig sse_kms_config;
sse_kms_config.key_id = "alias/test-key";
sse_kms_config.encryption_context = "arn:aws:s3:::bucket_ARN";
sse_kms_config.bucket_key_enabled = true;
testServerSideEncryption(
doWriteRequest,
"",
sse_kms_config,
"authorization: ... SignedHeaders="
"amz-sdk-invocation-id;"
"amz-sdk-request;"
"content-length;"
"content-md5;"
"content-type;"
"host;"
"x-amz-content-sha256;"
"x-amz-date;"
"x-amz-server-side-encryption;"
"x-amz-server-side-encryption-aws-kms-key-id;"
"x-amz-server-side-encryption-bucket-key-enabled;"
"x-amz-server-side-encryption-context, ...\n"
"x-amz-server-side-encryption: aws:kms\n"
"x-amz-server-side-encryption-aws-kms-key-id: alias/test-key\n"
"x-amz-server-side-encryption-bucket-key-enabled: true\n"
"x-amz-server-side-encryption-context: arn:aws:s3:::bucket_ARN\n");
}
#endif

View File

@ -70,6 +70,41 @@ namespace ErrorCodes
namespace S3
{
HTTPHeaderEntries getHTTPHeaders(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config)
{
HTTPHeaderEntries headers;
Poco::Util::AbstractConfiguration::Keys subconfig_keys;
config.keys(config_elem, subconfig_keys);
for (const std::string & subkey : subconfig_keys)
{
if (subkey.starts_with("header"))
{
auto header_str = config.getString(config_elem + "." + subkey);
auto delimiter = header_str.find(':');
if (delimiter == std::string::npos)
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Malformed s3 header value");
headers.emplace_back(header_str.substr(0, delimiter), header_str.substr(delimiter + 1, String::npos));
}
}
return headers;
}
ServerSideEncryptionKMSConfig getSSEKMSConfig(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config)
{
ServerSideEncryptionKMSConfig sse_kms_config;
if (config.has(config_elem + ".server_side_encryption_kms_key_id"))
sse_kms_config.key_id = config.getString(config_elem + ".server_side_encryption_kms_key_id");
if (config.has(config_elem + ".server_side_encryption_kms_encryption_context"))
sse_kms_config.encryption_context = config.getString(config_elem + ".server_side_encryption_kms_encryption_context");
if (config.has(config_elem + ".server_side_encryption_kms_bucket_key_enabled"))
sse_kms_config.bucket_key_enabled = config.getBool(config_elem + ".server_side_encryption_kms_bucket_key_enabled");
return sse_kms_config;
}
AuthSettings AuthSettings::loadFromConfig(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config)
{
auto access_key_id = config.getString(config_elem + ".access_key_id", "");
@ -93,26 +128,15 @@ AuthSettings AuthSettings::loadFromConfig(const std::string & config_elem, const
if (config.has(config_elem + ".no_sign_request"))
no_sign_request = config.getBool(config_elem + ".no_sign_request");
HTTPHeaderEntries headers;
Poco::Util::AbstractConfiguration::Keys subconfig_keys;
config.keys(config_elem, subconfig_keys);
for (const std::string & subkey : subconfig_keys)
{
if (subkey.starts_with("header"))
{
auto header_str = config.getString(config_elem + "." + subkey);
auto delimiter = header_str.find(':');
if (delimiter == std::string::npos)
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Malformed s3 header value");
headers.emplace_back(header_str.substr(0, delimiter), header_str.substr(delimiter + 1, String::npos));
}
}
HTTPHeaderEntries headers = getHTTPHeaders(config_elem, config);
ServerSideEncryptionKMSConfig sse_kms_config = getSSEKMSConfig(config_elem, config);
return AuthSettings
{
std::move(access_key_id), std::move(secret_access_key),
std::move(region),
std::move(server_side_encryption_customer_key_base64),
std::move(sse_kms_config),
std::move(headers),
use_environment_credentials,
use_insecure_imds_request,
@ -135,6 +159,7 @@ void AuthSettings::updateFrom(const AuthSettings & from)
headers = from.headers;
region = from.region;
server_side_encryption_customer_key_base64 = from.server_side_encryption_customer_key_base64;
server_side_encryption_kms_config = from.server_side_encryption_kms_config;
if (from.use_environment_credentials.has_value())
use_environment_credentials = from.use_environment_credentials;

View File

@ -1,5 +1,6 @@
#pragma once
#include <IO/S3/Client.h>
#include <IO/S3/PocoHTTPClient.h>
#include <IO/HTTPHeaderEntries.h>
@ -14,7 +15,6 @@
#include <Common/Exception.h>
#include <Common/Throttler_fwd.h>
#include <IO/S3/Client.h>
#include <IO/S3/URI.h>
#include <aws/core/Aws.h>
@ -71,6 +71,10 @@ namespace Poco::Util
namespace DB::S3
{
HTTPHeaderEntries getHTTPHeaders(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config);
ServerSideEncryptionKMSConfig getSSEKMSConfig(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config);
struct AuthSettings
{
static AuthSettings loadFromConfig(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config);
@ -79,6 +83,7 @@ struct AuthSettings
std::string secret_access_key;
std::string region;
std::string server_side_encryption_customer_key_base64;
ServerSideEncryptionKMSConfig server_side_encryption_kms_config;
HTTPHeaderEntries headers;

View File

@ -238,9 +238,8 @@ void WriteBufferFromS3::finalizeImpl()
}
}
void WriteBufferFromS3::createMultipartUpload()
void WriteBufferFromS3::fillCreateMultipartRequest(DB::S3::CreateMultipartUploadRequest & req)
{
DB::S3::CreateMultipartUploadRequest req;
req.SetBucket(bucket);
req.SetKey(key);
@ -250,6 +249,14 @@ void WriteBufferFromS3::createMultipartUpload()
if (object_metadata.has_value())
req.SetMetadata(object_metadata.value());
client_ptr->setKMSHeaders(req);
}
void WriteBufferFromS3::createMultipartUpload()
{
DB::S3::CreateMultipartUploadRequest req;
fillCreateMultipartRequest(req);
ProfileEvents::increment(ProfileEvents::S3CreateMultipartUpload);
if (write_settings.for_object_storage)
ProfileEvents::increment(ProfileEvents::DiskS3CreateMultipartUpload);
@ -571,6 +578,8 @@ void WriteBufferFromS3::fillPutRequest(S3::PutObjectRequest & req)
/// If we don't do it, AWS SDK can mistakenly set it to application/xml, see https://github.com/aws/aws-sdk-cpp/issues/1840
req.SetContentType("binary/octet-stream");
client_ptr->setKMSHeaders(req);
}
void WriteBufferFromS3::processPutRequest(const PutObjectTask & task)

View File

@ -61,6 +61,7 @@ private:
void processWithStrictParts();
void processWithDynamicParts();
void fillCreateMultipartRequest(S3::CreateMultipartUploadRequest & req);
void createMultipartUpload();
void writePart();
void completeMultipartUpload();

View File

@ -4,6 +4,7 @@
#include <Interpreters/Context.h>
#include <Interpreters/Access/InterpreterShowCreateAccessEntityQuery.h>
#include <Interpreters/Access/InterpreterShowGrantsQuery.h>
#include <Interpreters/formatWithPossiblyHidingSecrets.h>
#include <Columns/ColumnString.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <DataTypes/DataTypeString.h>
@ -33,7 +34,7 @@ QueryPipeline InterpreterShowAccessQuery::executeImpl() const
/// Build the result column.
MutableColumnPtr column = ColumnString::create();
for (const auto & query : queries)
column->insert(query->formatWithSecretsHidden());
column->insert(format({getContext(), *query}));
String desc = "ACCESS";
return QueryPipeline(std::make_shared<SourceFromSingleChunk>(Block{{std::move(column), std::make_shared<DataTypeString>(), desc}}));

View File

@ -1,4 +1,5 @@
#include <Interpreters/Access/InterpreterShowCreateAccessEntityQuery.h>
#include <Interpreters/formatWithPossiblyHidingSecrets.h>
#include <Parsers/Access/ASTShowCreateAccessEntityQuery.h>
#include <Parsers/Access/ASTCreateUserQuery.h>
#include <Parsers/Access/ASTCreateRoleQuery.h>
@ -254,7 +255,7 @@ QueryPipeline InterpreterShowCreateAccessEntityQuery::executeImpl()
/// Build the result column.
MutableColumnPtr column = ColumnString::create();
for (const auto & create_query : create_queries)
column->insert(create_query->formatWithSecretsHidden());
column->insert(format({getContext(), *create_query}));
/// Prepare description of the result column.
const auto & show_query = query_ptr->as<const ASTShowCreateAccessEntityQuery &>();

View File

@ -18,6 +18,7 @@
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/queryToString.h>
#include <Storages/IStorage.h>
#include <Common/CurrentThread.h>
#include <Common/SipHash.h>
#include <Common/FieldVisitorHash.h>
#include <Common/DateLUT.h>
@ -106,9 +107,10 @@ bool AsynchronousInsertQueue::InsertQuery::operator==(const InsertQuery & other)
return query_str == other.query_str && settings == other.settings;
}
AsynchronousInsertQueue::InsertData::Entry::Entry(String && bytes_, String && query_id_)
AsynchronousInsertQueue::InsertData::Entry::Entry(String && bytes_, String && query_id_, MemoryTracker * user_memory_tracker_)
: bytes(std::move(bytes_))
, query_id(std::move(query_id_))
, user_memory_tracker(user_memory_tracker_)
, create_time(std::chrono::system_clock::now())
{
}
@ -118,6 +120,15 @@ void AsynchronousInsertQueue::InsertData::Entry::finish(std::exception_ptr excep
if (finished.exchange(true))
return;
{
// To avoid races on counter of user's MemoryTracker we should free memory at this moment.
// Entries data must be destroyed in context of user who runs async insert.
// Each entry in the list may correspond to a different user,
// so we need to switch current thread's MemoryTracker.
UserMemoryTrackerSwitcher switcher(user_memory_tracker);
bytes = "";
}
if (exception_)
{
promise.set_exception(exception_);
@ -237,7 +248,7 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context)
if (auto quota = query_context->getQuota())
quota->used(QuotaType::WRITTEN_BYTES, bytes.size());
auto entry = std::make_shared<InsertData::Entry>(std::move(bytes), query_context->getCurrentQueryId());
auto entry = std::make_shared<InsertData::Entry>(std::move(bytes), query_context->getCurrentQueryId(), CurrentThread::getUserMemoryTracker());
InsertQuery key{query, settings};
InsertDataPtr data_to_process;
@ -445,6 +456,7 @@ try
{
auto buffer = std::make_unique<ReadBufferFromString>(entry->bytes);
current_entry = entry;
auto bytes_size = entry->bytes.size();
size_t num_rows = executor.execute(*buffer);
total_rows += num_rows;
chunk_info->offsets.push_back(total_rows);
@ -460,7 +472,7 @@ try
elem.event_time_microseconds = timeInMicroseconds(entry->create_time);
elem.query = key.query;
elem.query_id = entry->query_id;
elem.bytes = entry->bytes.size();
elem.bytes = bytes_size;
elem.rows = num_rows;
elem.exception = current_exception;
current_exception.clear();

View File

@ -1,6 +1,7 @@
#pragma once
#include <Parsers/IAST_fwd.h>
#include <Common/CurrentThread.h>
#include <Common/ThreadPool.h>
#include <Core/Settings.h>
#include <Poco/Logger.h>
@ -59,16 +60,42 @@ private:
UInt128 calculateHash() const;
};
struct UserMemoryTrackerSwitcher
{
explicit UserMemoryTrackerSwitcher(MemoryTracker * new_tracker)
{
auto * thread_tracker = CurrentThread::getMemoryTracker();
prev_untracked_memory = current_thread->untracked_memory;
prev_memory_tracker_parent = thread_tracker->getParent();
current_thread->untracked_memory = 0;
thread_tracker->setParent(new_tracker);
}
~UserMemoryTrackerSwitcher()
{
CurrentThread::flushUntrackedMemory();
auto * thread_tracker = CurrentThread::getMemoryTracker();
current_thread->untracked_memory = prev_untracked_memory;
thread_tracker->setParent(prev_memory_tracker_parent);
}
MemoryTracker * prev_memory_tracker_parent;
Int64 prev_untracked_memory;
};
struct InsertData
{
struct Entry
{
public:
const String bytes;
String bytes;
const String query_id;
MemoryTracker * const user_memory_tracker;
const std::chrono::time_point<std::chrono::system_clock> create_time;
Entry(String && bytes_, String && query_id_);
Entry(String && bytes_, String && query_id_, MemoryTracker * user_memory_tracker_);
void finish(std::exception_ptr exception_ = nullptr);
std::future<void> getFuture() { return promise.get_future(); }
@ -79,6 +106,19 @@ private:
std::atomic_bool finished = false;
};
~InsertData()
{
auto it = entries.begin();
// Entries must be destroyed in context of user who runs async insert.
// Each entry in the list may correspond to a different user,
// so we need to switch current thread's MemoryTracker parent on each iteration.
while (it != entries.end())
{
UserMemoryTrackerSwitcher switcher((*it)->user_memory_tracker);
it = entries.erase(it);
}
}
using EntryPtr = std::shared_ptr<Entry>;
std::list<EntryPtr> entries;

View File

@ -683,23 +683,25 @@ bool FileCache::tryReserve(FileSegment & file_segment, size_t size)
{
/// Space reservation is incremental, so file_segment_metadata is created first (with state empty),
/// and getQueueIterator() is assigned on first space reservation attempt.
file_segment.setQueueIterator(main_priority->add(
file_segment.getKeyMetadata(), file_segment.offset(), size, cache_lock));
queue_iterator = main_priority->add(file_segment.getKeyMetadata(), file_segment.offset(), size, cache_lock);
file_segment.setQueueIterator(queue_iterator);
}
file_segment.reserved_size += size;
chassert(file_segment.reserved_size == queue_iterator->getEntry().size);
if (query_context)
{
auto query_queue_it = query_context->tryGet(file_segment.key(), file_segment.offset(), cache_lock);
if (query_queue_it)
query_queue_it->updateSize(size);
else
query_context->add(file_segment, cache_lock);
query_context->add(file_segment.getKeyMetadata(), file_segment.offset(), size, cache_lock);
}
if (main_priority->getSize(cache_lock) > (1ull << 63))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache became inconsistent. There must be a bug");
file_segment.reserved_size += size;
return true;
}

View File

@ -28,7 +28,8 @@ IFileCachePriority::Iterator LRUFileCachePriority::add(
#ifndef NDEBUG
for (const auto & entry : queue)
{
if (entry.key == key && entry.offset == offset)
/// entry.size == 0 means entry was invalidated.
if (entry.size != 0 && entry.key == key && entry.offset == offset)
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Attempt to add duplicate queue entry to queue. (Key: {}, offset: {}, size: {})",
@ -52,7 +53,7 @@ IFileCachePriority::Iterator LRUFileCachePriority::add(
CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size);
CurrentMetrics::add(CurrentMetrics::FilesystemCacheElements);
LOG_TEST(log, "Added entry into LRU queue, key: {}, offset: {}", key, offset);
LOG_TEST(log, "Added entry into LRU queue, key: {}, offset: {}, size: {}", key, offset, size);
return std::make_shared<LRUFileCacheIterator>(this, iter);
}

View File

@ -200,6 +200,7 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata(
if (key_state == KeyMetadata::KeyState::REMOVING)
{
locked_metadata->removeFromCleanupQueue();
LOG_DEBUG(log, "Removal of key {} is cancelled", key);
return locked_metadata;
}
@ -265,6 +266,7 @@ void CacheMetadata::doCleanup()
locked_metadata->markAsRemoved();
erase(it);
LOG_DEBUG(log, "Key {} is removed from metadata", cleanup_key);
try
{
@ -287,7 +289,11 @@ void CacheMetadata::doCleanup()
LockedKey::LockedKey(std::shared_ptr<KeyMetadata> key_metadata_)
: key_metadata(key_metadata_)
, lock(key_metadata->guard.lock())
#ifdef ABORT_ON_LOGICAL_ERROR
, log(&Poco::Logger::get("LockedKey(" + key_metadata_->key.toString() + ")"))
#else
, log(&Poco::Logger::get("LockedKey"))
#endif
{
}
@ -297,6 +303,7 @@ LockedKey::~LockedKey()
return;
key_metadata->key_state = KeyMetadata::KeyState::REMOVING;
LOG_DEBUG(log, "Submitting key {} for removal", getKey());
key_metadata->cleanup_queue.add(getKey());
}
@ -377,6 +384,7 @@ void LockedKey::shrinkFileSegmentToDownloadedSize(
file_segment->getInfoForLogUnlocked(segment_lock));
}
chassert(file_segment->reserved_size >= downloaded_size);
int64_t diff = file_segment->reserved_size - downloaded_size;
metadata->file_segment = std::make_shared<FileSegment>(

View File

@ -65,22 +65,20 @@ FileCacheQueryLimit::QueryContext::QueryContext(
}
void FileCacheQueryLimit::QueryContext::add(
const FileSegment & file_segment,
KeyMetadataPtr key_metadata,
size_t offset,
size_t size,
const CacheGuard::Lock & lock)
{
const auto key = file_segment.key();
const auto offset = file_segment.offset();
auto it = getPriority().add(
file_segment.getKeyMetadata(), offset, file_segment.range().size(), lock);
auto [_, inserted] = records.emplace(FileCacheKeyAndOffset{key, offset}, it);
auto it = getPriority().add(key_metadata, offset, size, lock);
auto [_, inserted] = records.emplace(FileCacheKeyAndOffset{key_metadata->key, offset}, it);
if (!inserted)
{
it->remove(lock);
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Cannot add offset {} to query context under key {}, it already exists",
offset, key);
offset, key_metadata->key);
}
}

View File

@ -42,7 +42,9 @@ public:
const CacheGuard::Lock &);
void add(
const FileSegment & file_segment,
KeyMetadataPtr key_metadata,
size_t offset,
size_t size,
const CacheGuard::Lock &);
void remove(

View File

@ -1612,26 +1612,42 @@ StoragePtr Context::getViewSource() const
return view_source;
}
bool Context::displaySecretsInShowAndSelect() const
{
return shared->server_settings.display_secrets_in_show_and_select;
}
Settings Context::getSettings() const
{
auto lock = getLock();
return settings;
}
void Context::setSettings(const Settings & settings_)
{
auto lock = getLock();
auto old_readonly = settings.readonly;
auto old_allow_ddl = settings.allow_ddl;
auto old_allow_introspection_functions = settings.allow_introspection_functions;
const auto old_readonly = settings.readonly;
const auto old_allow_ddl = settings.allow_ddl;
const auto old_allow_introspection_functions = settings.allow_introspection_functions;
const auto old_display_secrets = settings.format_display_secrets_in_show_and_select;
settings = settings_;
if ((settings.readonly != old_readonly) || (settings.allow_ddl != old_allow_ddl) || (settings.allow_introspection_functions != old_allow_introspection_functions))
if ((settings.readonly != old_readonly)
|| (settings.allow_ddl != old_allow_ddl)
|| (settings.allow_introspection_functions != old_allow_introspection_functions)
|| (settings.format_display_secrets_in_show_and_select != old_display_secrets))
calculateAccessRights();
}
void Context::recalculateAccessRightsIfNeeded(std::string_view name)
{
if (name == "readonly"
|| name == "allow_ddl"
|| name == "allow_introspection_functions"
|| name == "format_display_secrets_in_show_and_select")
calculateAccessRights();
}
void Context::setSetting(std::string_view name, const String & value)
{
@ -1642,12 +1658,9 @@ void Context::setSetting(std::string_view name, const String & value)
return;
}
settings.set(name, value);
if (name == "readonly" || name == "allow_ddl" || name == "allow_introspection_functions")
calculateAccessRights();
recalculateAccessRightsIfNeeded(name);
}
void Context::setSetting(std::string_view name, const Field & value)
{
auto lock = getLock();
@ -1657,12 +1670,9 @@ void Context::setSetting(std::string_view name, const Field & value)
return;
}
settings.set(name, value);
if (name == "readonly" || name == "allow_ddl" || name == "allow_introspection_functions")
calculateAccessRights();
recalculateAccessRightsIfNeeded(name);
}
void Context::applySettingChange(const SettingChange & change)
{
try
@ -4274,7 +4284,6 @@ ReadSettings Context::getReadSettings() const
res.enable_filesystem_cache = settings.enable_filesystem_cache;
res.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache;
res.enable_filesystem_cache_log = settings.enable_filesystem_cache_log;
res.enable_filesystem_cache_on_lower_level = settings.enable_filesystem_cache_on_lower_level;
res.filesystem_cache_max_download_size = settings.filesystem_cache_max_download_size;
res.skip_download_if_exceeds_query_cache = settings.skip_download_if_exceeds_query_cache;

View File

@ -683,6 +683,7 @@ public:
MultiVersion<Macros>::Version getMacros() const;
void setMacros(std::unique_ptr<Macros> && macros);
bool displaySecretsInShowAndSelect() const;
Settings getSettings() const;
void setSettings(const Settings & settings_);
@ -1145,6 +1146,7 @@ private:
/// Compute and set actual user settings, client_info.current_user should be set
void calculateAccessRights();
void recalculateAccessRightsIfNeeded(std::string_view setting_name);
template <typename... Args>
void checkAccessImpl(const Args &... args) const;

View File

@ -9,6 +9,7 @@
#include <Common/typeid_cast.h>
#include <Access/Common/AccessFlags.h>
#include <Interpreters/Context.h>
#include <Interpreters/formatWithPossiblyHidingSecrets.h>
#include <Interpreters/InterpreterShowCreateQuery.h>
#include <Parsers/ASTCreateQuery.h>
@ -94,10 +95,8 @@ QueryPipeline InterpreterShowCreateQuery::executeImpl()
create.to_inner_uuid = UUIDHelpers::Nil;
}
String res = create_query->formatWithSecretsHidden(/* max_length= */ 0, /* one_line= */ false);
MutableColumnPtr column = ColumnString::create();
column->insert(res);
column->insert(format({.ctx = getContext(), .query = *create_query, .one_line = false}));
return QueryPipeline(std::make_shared<SourceFromSingleChunk>(Block{{
std::move(column),

View File

@ -0,0 +1,25 @@
#pragma once
#include "Access/ContextAccess.h"
#include "Interpreters/Context.h"
namespace DB
{
struct SecretHidingFormatSettings
{
// We can't store const Context& as there's a dangerous usage {.ctx = *getContext()}
// which is UB in case getContext()'s return ptr is the only one holding the object
const ContextPtr & ctx;
const IAST & query;
size_t max_length = 0;
bool one_line = true;
};
inline String format(const SecretHidingFormatSettings & settings)
{
const bool show_secrets = settings.ctx->displaySecretsInShowAndSelect()
&& settings.ctx->getSettingsRef().format_display_secrets_in_show_and_select
&& settings.ctx->getAccess()->isGranted(AccessType::displaySecretsInShowAndSelect);
return settings.query.formatWithPossiblyHidingSensitiveData(settings.max_length, settings.one_line, show_secrets);
}
}

View File

@ -167,14 +167,10 @@ size_t IAST::checkDepthImpl(size_t max_depth) const
return res;
}
String IAST::formatWithSecretsHidden(size_t max_length, bool one_line) const
String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const
{
WriteBufferFromOwnString buf;
FormatSettings settings{buf, one_line};
settings.show_secrets = false;
format(settings);
format({buf, one_line, show_secrets});
return wipeSensitiveDataAndCutToLength(buf.str(), max_length);
}

View File

@ -200,8 +200,8 @@ public:
// Newline or whitespace.
char nl_or_ws;
FormatSettings(WriteBuffer & ostr_, bool one_line_)
: ostr(ostr_), one_line(one_line_)
FormatSettings(WriteBuffer & ostr_, bool one_line_, bool show_secrets_ = true)
: ostr(ostr_), one_line(one_line_), show_secrets(show_secrets_)
{
nl_or_ws = one_line ? ' ' : '\n';
}
@ -251,12 +251,26 @@ public:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown element in AST: {}", getID());
}
// A simple way to add some user-readable context to an error message.
String formatWithSecretsHidden(size_t max_length = 0, bool one_line = true) const;
String formatForLogging(size_t max_length = 0) const { return formatWithSecretsHidden(max_length, true); }
String formatForErrorMessage() const { return formatWithSecretsHidden(0, true); }
// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied.
// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience.
String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const;
/*
* formatForLogging and formatForErrorMessage always hide secrets. This inconsistent
* behaviour is due to the fact such functions are called from Client which knows nothing about
* access rights and settings. Moreover, the only use case for displaying secrets are backups,
* and backup tools use only direct input and ignore logs and error messages.
*/
String formatForLogging(size_t max_length = 0) const
{
return formatWithPossiblyHidingSensitiveData(max_length, true, false);
}
String formatForErrorMessage() const
{
return formatWithPossiblyHidingSensitiveData(0, true, false);
}
/// If an AST has secret parts then formatForLogging() will replace them with the placeholder '[HIDDEN]'.
virtual bool hasSecretParts() const { return childrenHaveSecretParts(); }
void cloneChildren();

View File

@ -244,7 +244,9 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
auto default_function = std::make_shared<ASTFunction>();
default_function->name = "defaultValueOfTypeName";
default_function->arguments = std::make_shared<ASTExpressionList>();
default_function->arguments->children.emplace_back(std::make_shared<ASTLiteral>(type->as<ASTFunction>()->formatWithSecretsHidden()));
// Ephemeral columns don't really have secrets but we need to format
// into a String, hence the strange call
default_function->arguments->children.emplace_back(std::make_shared<ASTLiteral>(type->as<ASTFunction>()->formatForLogging()));
default_expression = default_function;
}

View File

@ -134,6 +134,7 @@ void registerOutputFormatArrow(FormatFactory & factory)
return std::make_shared<ArrowBlockOutputFormat>(buf, sample, true, format_settings);
});
factory.markFormatHasNoAppendSupport("ArrowStream");
factory.markOutputFormatPrefersLargeBlocks("ArrowStream");
}
}

View File

@ -624,6 +624,7 @@ void registerOutputFormatORC(FormatFactory & factory)
return std::make_shared<ORCBlockOutputFormat>(buf, sample, format_settings);
});
factory.markFormatHasNoAppendSupport("ORC");
factory.markOutputFormatPrefersLargeBlocks("ORC");
}
}

View File

@ -70,6 +70,67 @@ ParquetBlockOutputFormat::ParquetBlockOutputFormat(WriteBuffer & out_, const Blo
}
void ParquetBlockOutputFormat::consume(Chunk chunk)
{
/// Do something like SquashingTransform to produce big enough row groups.
/// Because the real SquashingTransform is only used for INSERT, not for SELECT ... INTO OUTFILE.
/// The latter doesn't even have a pipeline where a transform could be inserted, so it's more
/// convenient to do the squashing here.
appendToAccumulatedChunk(std::move(chunk));
if (!accumulated_chunk)
return;
const size_t target_rows = std::max(static_cast<UInt64>(1), format_settings.parquet.row_group_rows);
if (accumulated_chunk.getNumRows() < target_rows &&
accumulated_chunk.bytes() < format_settings.parquet.row_group_bytes)
return;
/// Increase row group size slightly (by < 2x) to avoid adding a small row groups for the
/// remainder of the new chunk.
/// E.g. suppose input chunks are 70K rows each, and max_rows = 1M. Then we'll have
/// getNumRows() = 1.05M. We want to write all 1.05M as one row group instead of 1M and 0.05M.
size_t num_row_groups = std::max(static_cast<UInt64>(1), accumulated_chunk.getNumRows() / target_rows);
size_t row_group_size = (accumulated_chunk.getNumRows() - 1) / num_row_groups + 1; // round up
write(std::move(accumulated_chunk), row_group_size);
accumulated_chunk.clear();
}
void ParquetBlockOutputFormat::finalizeImpl()
{
if (accumulated_chunk)
write(std::move(accumulated_chunk), format_settings.parquet.row_group_rows);
if (!file_writer)
{
const Block & header = getPort(PortKind::Main).getHeader();
write(Chunk(header.getColumns(), 0), 1);
}
auto status = file_writer->Close();
if (!status.ok())
throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while closing a table: {}", status.ToString());
}
void ParquetBlockOutputFormat::resetFormatterImpl()
{
file_writer.reset();
}
void ParquetBlockOutputFormat::appendToAccumulatedChunk(Chunk chunk)
{
if (!accumulated_chunk)
{
accumulated_chunk = std::move(chunk);
return;
}
chassert(accumulated_chunk.getNumColumns() == chunk.getNumColumns());
accumulated_chunk.append(chunk);
}
void ParquetBlockOutputFormat::write(Chunk chunk, size_t row_group_size)
{
const size_t columns_num = chunk.getNumColumns();
std::shared_ptr<arrow::Table> arrow_table;
@ -105,32 +166,12 @@ void ParquetBlockOutputFormat::consume(Chunk chunk)
file_writer = std::move(result.ValueOrDie());
}
// TODO: calculate row_group_size depending on a number of rows and table size
auto status = file_writer->WriteTable(*arrow_table, format_settings.parquet.row_group_size);
auto status = file_writer->WriteTable(*arrow_table, row_group_size);
if (!status.ok())
throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while writing a table: {}", status.ToString());
}
void ParquetBlockOutputFormat::finalizeImpl()
{
if (!file_writer)
{
const Block & header = getPort(PortKind::Main).getHeader();
consume(Chunk(header.getColumns(), 0));
}
auto status = file_writer->Close();
if (!status.ok())
throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while closing a table: {}", status.ToString());
}
void ParquetBlockOutputFormat::resetFormatterImpl()
{
file_writer.reset();
}
void registerOutputFormatParquet(FormatFactory & factory)
{
factory.registerOutputFormat(

View File

@ -35,6 +35,8 @@ public:
private:
void consume(Chunk) override;
void appendToAccumulatedChunk(Chunk chunk);
void write(Chunk chunk, size_t row_group_size);
void finalizeImpl() override;
void resetFormatterImpl() override;
@ -42,6 +44,8 @@ private:
std::unique_ptr<parquet::arrow::FileWriter> file_writer;
std::unique_ptr<CHColumnToArrowColumn> ch_column_to_arrow_column;
Chunk accumulated_chunk;
};
}

View File

@ -582,6 +582,9 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
ContextPtr context = reading->getContext();
MergeTreeDataSelectExecutor reader(reading->getMergeTreeData());
auto ordinary_reading_select_result = reading->selectRangesToRead(parts);
size_t ordinary_reading_marks = ordinary_reading_select_result->marks();
/// Selecting best candidate.
for (auto & candidate : candidates.real)
{
@ -597,12 +600,18 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
if (!analyzed)
continue;
if (candidate.sum_marks > ordinary_reading_marks)
continue;
if (best_candidate == nullptr || best_candidate->sum_marks > candidate.sum_marks)
best_candidate = &candidate;
}
if (!best_candidate)
{
reading->setAnalyzedResult(std::move(ordinary_reading_select_result));
return false;
}
QueryPlanStepPtr projection_reading;
bool has_ordinary_parts;

View File

@ -44,6 +44,8 @@ public:
// Must insert the result for current_row.
virtual void windowInsertResultInto(const WindowTransform * transform,
size_t function_index) = 0;
virtual std::optional<WindowFrame> getDefaultFrame() const { return {}; }
};
// Compares ORDER BY column values at given rows to find the boundaries of frame:
@ -222,6 +224,15 @@ WindowTransform::WindowTransform(const Block & input_header_,
/// Currently we have slightly wrong mixup of the interfaces of Window and Aggregate functions.
workspace.window_function_impl = dynamic_cast<IWindowFunction *>(const_cast<IAggregateFunction *>(aggregate_function.get()));
/// Some functions may have non-standard default frame.
/// Use it if it's the only function over the current window.
if (window_description.frame.is_default && functions.size() == 1 && workspace.window_function_impl)
{
auto custom_default_frame = workspace.window_function_impl->getDefaultFrame();
if (custom_default_frame)
window_description.frame = *custom_default_frame;
}
workspace.aggregate_function_state.reset(
aggregate_function->sizeOfData(),
aggregate_function->alignOfData());
@ -1977,18 +1988,23 @@ struct WindowFunctionNtile final : public WindowFunction
: WindowFunction(name_, argument_types_, parameters_, std::make_shared<DataTypeUInt64>())
{
if (argument_types.size() != 1)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} takes exactly one parameter", name_);
}
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} takes exactly one argument", name_);
auto type_id = argument_types[0]->getTypeId();
if (type_id != TypeIndex::UInt8 && type_id != TypeIndex::UInt16 && type_id != TypeIndex::UInt32 && type_id != TypeIndex::UInt64)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ntile's argument type must be an unsigned integer (not larger then 64-bit), but got {}", argument_types[0]->getName());
}
throw Exception(ErrorCodes::BAD_ARGUMENTS, "'{}' argument type must be an unsigned integer (not larger than 64-bit), got {}", name_, argument_types[0]->getName());
}
bool allocatesMemoryInArena() const override { return false; }
std::optional<WindowFrame> getDefaultFrame() const override
{
WindowFrame frame;
frame.type = WindowFrame::FrameType::ROWS;
frame.end_type = WindowFrame::BoundaryType::Unbounded;
return frame;
}
void windowInsertResultInto(const WindowTransform * transform,
size_t function_index) override
{
@ -1999,7 +2015,7 @@ struct WindowFunctionNtile final : public WindowFunction
const auto & workspace = transform->workspaces[function_index];
const auto & arg_col = *current_block.original_input_columns[workspace.argument_column_indices[0]];
if (!isColumnConst(arg_col))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ntile's argument must be a constant");
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of 'ntile' function must be a constant");
auto type_id = argument_types[0]->getTypeId();
if (type_id == TypeIndex::UInt8)
buckets = arg_col[transform->current_row.row].get<UInt8>();
@ -2012,7 +2028,7 @@ struct WindowFunctionNtile final : public WindowFunction
if (!buckets)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ntile's argument must be greater than 0");
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of 'ntile' funtcion must be greater than zero");
}
}
// new partition
@ -2090,22 +2106,16 @@ private:
static void checkWindowFrameType(const WindowTransform * transform)
{
if (transform->order_by_indices.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ntile's window frame must have order by clause");
if (transform->window_description.frame.type != WindowFrame::FrameType::ROWS)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ntile's frame type must be ROWS");
}
if (transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ntile's frame start type must be UNBOUNDED PRECEDING");
}
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Window frame for 'ntile' function must have ORDER BY clause");
if (transform->window_description.frame.end_type != WindowFrame::BoundaryType::Unbounded)
// We must wait all for the partition end and get the total rows number in this
// partition. So before the end of this partition, there is no any block could be
// dropped out.
bool is_frame_supported = transform->window_description.frame.begin_type == WindowFrame::BoundaryType::Unbounded
&& transform->window_description.frame.end_type == WindowFrame::BoundaryType::Unbounded;
if (!is_frame_supported)
{
// We must wait all for the partition end and get the total rows number in this
// partition. So before the end of this partition, there is no any block could be
// dropped out.
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ntile's frame end type must be UNBOUNDED FOLLOWING");
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Window frame for function 'ntile' should be 'ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING'");
}
}
};

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