Merge remote-tracking branch 'rschu1ze/master' into clang-17

This commit is contained in:
Robert Schulze 2023-09-21 14:45:55 +00:00
commit 5209bd2d51
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
160 changed files with 1948 additions and 447 deletions

View File

@ -31,7 +31,7 @@
* Add new functions `structureToCapnProtoSchema`/`structureToProtobufSchema` that convert ClickHouse table structure to CapnProto/Protobuf format schema. Allow to input/output data in CapnProto/Protobuf format without external format schema using autogenerated schema from table structure (controled by settings `format_capn_proto_use_autogenerated_schema`/`format_protobuf_use_autogenerated_schema`). Allow to export autogenerated schema while input/outoput using setting `output_format_schema`. [#52278](https://github.com/ClickHouse/ClickHouse/pull/52278) ([Kruglov Pavel](https://github.com/Avogar)).
* A new field `query_cache_usage` in `system.query_log` now shows if and how the query cache was used. [#52384](https://github.com/ClickHouse/ClickHouse/pull/52384) ([Robert Schulze](https://github.com/rschu1ze)).
* Add new function `startsWithUTF8` and `endsWithUTF8`. [#52555](https://github.com/ClickHouse/ClickHouse/pull/52555) ([李扬](https://github.com/taiyang-li)).
* Allow variable number of columns in TSV/CuatomSeprarated/JSONCompactEachRow, make schema inference work with variable number of columns. Add settings `input_format_tsv_allow_variable_number_of_columns`, `input_format_custom_allow_variable_number_of_columns`, `input_format_json_compact_allow_variable_number_of_columns`. [#52692](https://github.com/ClickHouse/ClickHouse/pull/52692) ([Kruglov Pavel](https://github.com/Avogar)).
* Allow variable number of columns in TSV/CustomSeparated/JSONCompactEachRow, make schema inference work with variable number of columns. Add settings `input_format_tsv_allow_variable_number_of_columns`, `input_format_custom_allow_variable_number_of_columns`, `input_format_json_compact_allow_variable_number_of_columns`. [#52692](https://github.com/ClickHouse/ClickHouse/pull/52692) ([Kruglov Pavel](https://github.com/Avogar)).
* Added `SYSTEM STOP/START PULLING REPLICATION LOG` queries (for testing `ReplicatedMergeTree`). [#52881](https://github.com/ClickHouse/ClickHouse/pull/52881) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Allow to execute constant non-deterministic functions in mutations on initiator. [#53129](https://github.com/ClickHouse/ClickHouse/pull/53129) ([Anton Popov](https://github.com/CurtizJ)).
* Add input format `One` that doesn't read any data and always returns single row with column `dummy` with type `UInt8` and value `0` like `system.one`. It can be used together with `_file/_path` virtual columns to list files in file/s3/url/hdfs/etc table functions without reading any data. [#53209](https://github.com/ClickHouse/ClickHouse/pull/53209) ([Kruglov Pavel](https://github.com/Avogar)).

View File

@ -31,7 +31,11 @@ RUN mkdir -p /tmp/clickhouse-odbc-tmp \
&& cp /tmp/clickhouse-odbc-tmp/lib64/*.so /usr/local/lib/ \
&& odbcinst -i -d -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbcinst.ini.sample \
&& odbcinst -i -s -l -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbc.ini.sample \
&& rm -rf /tmp/clickhouse-odbc-tmp
&& rm -rf /tmp/clickhouse-odbc-tmp \
&& mkdir -p /var/lib/clickhouse \
&& chmod 777 /var/lib/clickhouse
# chmod 777 to make the container user independent
ENV TZ=Europe/Amsterdam
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone

View File

@ -28,6 +28,12 @@ FASTTEST_BUILD=$(readlink -f "${FASTTEST_BUILD:-${BUILD:-$FASTTEST_WORKSPACE/bui
FASTTEST_DATA=$(readlink -f "${FASTTEST_DATA:-$FASTTEST_WORKSPACE/db-fasttest}")
FASTTEST_OUTPUT=$(readlink -f "${FASTTEST_OUTPUT:-$FASTTEST_WORKSPACE}")
PATH="$FASTTEST_BUILD/programs:$FASTTEST_SOURCE/tests:$PATH"
# Work around for non-existent user
if [ "$HOME" == "/" ]; then
HOME="$FASTTEST_WORKSPACE/user-home"
mkdir -p "$HOME"
export HOME
fi
# Export these variables, so that all subsequent invocations of the script
# use them, and not try to guess them anew, which leads to weird effects.
@ -152,7 +158,11 @@ function clone_submodules
)
git submodule sync
git submodule update --jobs=16 --depth 1 --single-branch --init "${SUBMODULES_TO_UPDATE[@]}"
git submodule init
# --jobs does not work as fast as real parallel running
printf '%s\0' "${SUBMODULES_TO_UPDATE[@]}" | \
xargs --max-procs=100 --null --no-run-if-empty --max-args=1 \
git submodule update --depth 1 --single-branch
git submodule foreach git reset --hard
git submodule foreach git checkout @ -f
git submodule foreach git clean -xfd
@ -271,34 +281,12 @@ case "$stage" in
;&
"clone_root")
clone_root
# Pass control to the script from cloned sources, unless asked otherwise.
if ! [ -v FASTTEST_LOCAL_SCRIPT ]
then
# 'run' stage is deprecated, used for compatibility with old scripts.
# Replace with 'clone_submodules' after Nov 1, 2020.
# cd and CLICKHOUSE_DIR are also a setup for old scripts, remove as well.
# In modern script we undo it by changing back into workspace dir right
# away, see below. Remove that as well.
cd "$FASTTEST_SOURCE"
CLICKHOUSE_DIR=$(pwd)
export CLICKHOUSE_DIR
stage=run "$FASTTEST_SOURCE/docker/test/fasttest/run.sh"
exit $?
fi
;&
"run")
# A deprecated stage that is called by old script and equivalent to everything
# after cloning root, starting with cloning submodules.
;&
"clone_submodules")
# Recover after being called from the old script that changes into source directory.
# See the compatibility hacks in `clone_root` stage above. Remove at the same time,
# after Nov 1, 2020.
cd "$FASTTEST_WORKSPACE"
clone_submodules 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/submodule_log.txt"
;&
"run_cmake")
cd "$FASTTEST_WORKSPACE"
run_cmake
;&
"build")

View File

@ -203,9 +203,10 @@ Parameter `NumTrees` is the number of trees which the algorithm creates (default
more accurate search results but slower index creation / query times (approximately linearly) as well as larger index sizes.
:::note
Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays **must** have same length. Use
[CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1
CHECK length(vectors) = 256`.
Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays must have same length. To avoid
errors, you can use a [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints), for example, `CONSTRAINT
constraint_name_1 CHECK length(vectors) = 256`. Also, unspecified `Array` values in INSERT statements (i.e. default values) are not
supported.
:::
Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many tree nodes are inspected during SELECTs. Larger
@ -223,6 +224,7 @@ SETTINGS annoy_index_search_k_nodes=100;
The Annoy index currently does not work with per-table, non-default `index_granularity` settings (see
[here](https://github.com/ClickHouse/ClickHouse/pull/51325#issuecomment-1605920475)). If necessary, the value must be changed in config.xml.
:::
## USearch {#usearch}
This type of ANN index is based on the [the USearch library](https://github.com/unum-cloud/usearch), which implements the [HNSW

View File

@ -1354,3 +1354,4 @@ In this sample configuration:
- `_part_uuid` — Unique part identifier (if enabled MergeTree setting `assign_part_uuids`).
- `_partition_value` — Values (a tuple) of a `partition by` expression.
- `_sample_factor` — Sample factor (from the query).
- `_block_number` — Block number of the row, it is persisted on merges when `allow_experimental_block_number_column` is set to true.

View File

@ -142,7 +142,7 @@ As a result, the query cache stores for each query multiple (partial)
result blocks. While this behavior is a good default, it can be suppressed using setting
[query_cache_squash_partial_results](settings/settings.md#query-cache-squash-partial-results).
Also, results of queries with non-deterministic functions are not cached. Such functions include
Also, results of queries with non-deterministic functions are not cached by default. Such functions include
- functions for accessing dictionaries: [`dictGet()`](../sql-reference/functions/ext-dict-functions.md#dictGet) etc.
- [user-defined functions](../sql-reference/statements/create/function.md),
- functions which return the current date or time: [`now()`](../sql-reference/functions/date-time-functions.md#now),
@ -158,7 +158,7 @@ Also, results of queries with non-deterministic functions are not cached. Such f
- functions which depend on the environment: [`currentUser()`](../sql-reference/functions/other-functions.md#currentUser),
[`queryID()`](../sql-reference/functions/other-functions.md#queryID),
[`getMacro()`](../sql-reference/functions/other-functions.md#getMacro) etc.
Caching of non-deterministic functions can be forced regardless using setting
To force caching of results of queries with non-deterministic functions regardless, use setting
[query_cache_store_results_of_queries_with_nondeterministic_functions](settings/settings.md#query-cache-store-results-of-queries-with-nondeterministic-functions).
Finally, entries in the query cache are not shared between users due to security reasons. For example, user A must not be able to bypass a

View File

@ -854,3 +854,9 @@ Possible values:
- `Always` or `Never`.
Default value: `Never`
## allow_experimental_block_number_column
Persists virtual column `_block_number` on merges.
Default value: false.

View File

@ -4,7 +4,7 @@ sidebar_position: 52
sidebar_label: Array(T)
---
# Array(t)
# Array(T)
An array of `T`-type items, with the starting array index as 1. `T` can be any data type, including an array.

View File

@ -21,6 +21,7 @@ DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] [SYNC]
## DROP TABLE
Deletes the table.
In case when `IF EMPTY` clause is specified server will check if table is empty only on replica that received initial query.
:::tip
Also see [UNDROP TABLE](/docs/en/sql-reference/statements/undrop.md)
@ -29,7 +30,7 @@ Also see [UNDROP TABLE](/docs/en/sql-reference/statements/undrop.md)
Syntax:
``` sql
DROP [TEMPORARY] TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] [SYNC]
DROP [TEMPORARY] TABLE [IF EXISTS] [IF EMPTY] [db.]name [ON CLUSTER cluster] [SYNC]
```
## DROP DICTIONARY

View File

@ -132,13 +132,6 @@ func TestConfigFileFrameCopy(t *testing.T) {
configFrame, errs := data.NewConfigFileFrame(path.Join(cwd, "../../../testdata", "configs", "xml"))
require.Empty(t, errs)
i := 0
sizes := map[string]int64{
"users.xml": int64(2017),
"default-password.xml": int64(188),
"config.xml": int64(59377),
"server-include.xml": int64(168),
"user-include.xml": int64(559),
}
var checkedFiles []string
for {
values, ok, err := configFrame.Next()
@ -153,8 +146,6 @@ func TestConfigFileFrameCopy(t *testing.T) {
newPath := path.Join(tmrDir, fileName)
err = configFile.Copy(newPath, true)
require.FileExists(t, newPath)
destInfo, _ := os.Stat(newPath)
require.Equal(t, sizes[fileName], destInfo.Size())
require.Nil(t, err)
bytes, err := ioutil.ReadFile(newPath)
require.Nil(t, err)
@ -186,13 +177,6 @@ func TestConfigFileFrameCopy(t *testing.T) {
configFrame, errs := data.NewConfigFileFrame(path.Join(cwd, "../../../testdata", "configs", "yaml"))
require.Empty(t, errs)
i := 0
sizes := map[string]int64{
"users.yaml": int64(1023),
"default-password.yaml": int64(132),
"config.yaml": int64(41633),
"server-include.yaml": int64(21),
"user-include.yaml": int64(120),
}
var checkedFiles []string
for {
values, ok, err := configFrame.Next()
@ -207,8 +191,6 @@ func TestConfigFileFrameCopy(t *testing.T) {
newPath := path.Join(tmrDir, fileName)
err = configFile.Copy(newPath, true)
require.FileExists(t, newPath)
destInfo, _ := os.Stat(newPath)
require.Equal(t, sizes[fileName], destInfo.Size())
require.Nil(t, err)
bytes, err := ioutil.ReadFile(newPath)
require.Nil(t, err)

View File

@ -57,7 +57,7 @@ public:
String relative_path_from = validatePathAndGetAsRelative(path_from);
String relative_path_to = validatePathAndGetAsRelative(path_to);
disk_from->copyDirectoryContent(relative_path_from, disk_to, relative_path_to, /* settings= */ {});
disk_from->copyDirectoryContent(relative_path_from, disk_to, relative_path_to, /* read_settings= */ {}, /* write_settings= */ {});
}
};
}

View File

@ -32,6 +32,8 @@
#include <Common/randomSeed.h>
#include <Common/ThreadPool.h>
#include <Loggers/Loggers.h>
#include <Loggers/OwnFormattingChannel.h>
#include <Loggers/OwnPatternFormatter.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromFileDescriptor.h>
@ -599,7 +601,9 @@ void LocalServer::processConfig()
{
auto poco_logs_level = Poco::Logger::parseLevel(level);
Poco::Logger::root().setLevel(poco_logs_level);
Poco::Logger::root().setChannel(Poco::AutoPtr<Poco::SimpleFileChannel>(new Poco::SimpleFileChannel(server_logs_file)));
Poco::AutoPtr<OwnPatternFormatter> pf = new OwnPatternFormatter;
Poco::AutoPtr<OwnFormattingChannel> log = new OwnFormattingChannel(pf, new Poco::SimpleFileChannel(server_logs_file));
Poco::Logger::root().setChannel(log);
logging_initialized = true;
}
else if (logging || is_interactive)

View File

@ -46,7 +46,7 @@ void BackupReaderDisk::copyFileToDisk(const String & path_in_backup, size_t file
{
/// Use more optimal way.
LOG_TRACE(log, "Copying file {} from disk {} to disk {}", path_in_backup, disk->getName(), destination_disk->getName());
disk->copyFile(root_path / path_in_backup, *destination_disk, destination_path, write_settings);
disk->copyFile(root_path / path_in_backup, *destination_disk, destination_path, read_settings, write_settings);
return; /// copied!
}
}
@ -119,7 +119,7 @@ void BackupWriterDisk::copyFileFromDisk(const String & path_in_backup, DiskPtr s
LOG_TRACE(log, "Copying file {} from disk {} to disk {}", src_path, src_disk->getName(), disk->getName());
auto dest_file_path = root_path / path_in_backup;
disk->createDirectories(dest_file_path.parent_path());
src_disk->copyFile(src_path, *disk, dest_file_path, write_settings);
src_disk->copyFile(src_path, *disk, dest_file_path, read_settings, write_settings);
return; /// copied!
}
}

View File

@ -170,6 +170,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s
/* dest_bucket= */ blob_path[1],
/* dest_key= */ blob_path[0],
request_settings,
read_settings,
object_attributes,
threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupReaderS3"),
/* for_disk_s3= */ true);
@ -230,6 +231,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src
s3_uri.bucket,
fs::path(s3_uri.key) / path_in_backup,
request_settings,
read_settings,
{},
threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupWriterS3"));
return; /// copied!

View File

@ -585,7 +585,9 @@
M(700, USER_SESSION_LIMIT_EXCEEDED) \
M(701, CLUSTER_DOESNT_EXIST) \
M(702, CLIENT_INFO_DOES_NOT_MATCH) \
\
M(703, INVALID_IDENTIFIER) \
M(704, CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS) \
M(705, TABLE_NOT_EMPTY) \
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \
M(1001, STD_EXCEPTION) \

View File

@ -103,6 +103,7 @@ static const size_t signal_pipe_buf_size =
+ sizeof(siginfo_t)
+ sizeof(ucontext_t*)
+ sizeof(StackTrace)
+ sizeof(UInt64)
+ sizeof(UInt32)
+ sizeof(void*);

231
src/DataTypes/Utils.cpp Normal file
View File

@ -0,0 +1,231 @@
#include <DataTypes/Utils.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeTuple.h>
namespace DB
{
bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_type)
{
auto from_which_type = WhichDataType(from_type->getTypeId());
bool to_type_was_nullable = isNullableOrLowCardinalityNullable(to_type);
auto to_type_unwrapped = removeNullable(removeLowCardinality(to_type));
if (from_type->equals(*to_type_unwrapped))
return true;
auto to_which_type = WhichDataType(to_type_unwrapped->getTypeId());
switch (from_which_type.idx)
{
case TypeIndex::UInt8:
case TypeIndex::UInt16:
case TypeIndex::UInt32:
case TypeIndex::UInt64:
case TypeIndex::UInt128:
case TypeIndex::UInt256:
{
if (to_which_type.isUInt() &&
to_type_unwrapped->getSizeOfValueInMemory() >= from_type->getSizeOfValueInMemory())
return true;
if (to_which_type.isString())
return true;
return false;
}
case TypeIndex::Int8:
case TypeIndex::Int16:
case TypeIndex::Int32:
case TypeIndex::Int64:
case TypeIndex::Int128:
case TypeIndex::Int256:
{
if (to_which_type.isInt() &&
to_type_unwrapped->getSizeOfValueInMemory() >= from_type->getSizeOfValueInMemory())
return true;
if (to_which_type.isString())
return true;
return false;
}
case TypeIndex::Float32:
{
if (to_which_type.isFloat64() || to_which_type.isString())
return true;
return false;
}
case TypeIndex::Float64:
case TypeIndex::Date:
case TypeIndex::Date32:
case TypeIndex::DateTime:
case TypeIndex::DateTime64:
case TypeIndex::FixedString:
case TypeIndex::Enum8:
case TypeIndex::Enum16:
case TypeIndex::IPv6:
{
if (to_which_type.isString())
return true;
return false;
}
case TypeIndex::Decimal32:
case TypeIndex::Decimal64:
case TypeIndex::Decimal128:
case TypeIndex::Decimal256:
{
if (to_which_type.isDecimal())
{
auto from_type_decimal_precision = getDecimalPrecision(*from_type);
auto to_type_decimal_precision = getDecimalPrecision(*to_type_unwrapped);
if (from_type_decimal_precision > to_type_decimal_precision)
return false;
auto from_type_decimal_scale = getDecimalScale(*from_type);
auto to_type_decimal_scale = getDecimalScale(*to_type_unwrapped);
if (from_type_decimal_scale > to_type_decimal_scale)
return false;
return true;
}
if (to_which_type.isString())
return true;
return false;
}
case TypeIndex::UUID:
{
if (to_which_type.isUInt128() || to_which_type.isString())
return true;
return false;
}
case TypeIndex::IPv4:
{
if (to_which_type.isUInt32() || to_which_type.isUInt64() || to_which_type.isString())
return true;
return false;
}
case TypeIndex::Nullable:
{
if (to_type_was_nullable)
{
const auto & from_type_nullable = assert_cast<const DataTypeNullable &>(*from_type);
return canBeSafelyCasted(from_type_nullable.getNestedType(), to_type_unwrapped);
}
if (to_which_type.isString())
return true;
return false;
}
case TypeIndex::LowCardinality:
{
const auto & from_type_low_cardinality = assert_cast<const DataTypeLowCardinality &>(*from_type);
return canBeSafelyCasted(from_type_low_cardinality.getDictionaryType(), to_type_unwrapped);
}
case TypeIndex::Array:
{
if (to_which_type.isArray())
{
const auto & from_type_array = assert_cast<const DataTypeArray &>(*from_type);
const auto & to_type_array = assert_cast<const DataTypeArray &>(*to_type_unwrapped);
return canBeSafelyCasted(from_type_array.getNestedType(), to_type_array.getNestedType());
}
if (to_which_type.isString())
return true;
return false;
}
case TypeIndex::Map:
{
if (to_which_type.isMap())
{
const auto & from_type_map = assert_cast<const DataTypeMap &>(*from_type);
const auto & to_type_map = assert_cast<const DataTypeMap &>(*to_type_unwrapped);
if (!canBeSafelyCasted(from_type_map.getKeyType(), to_type_map.getKeyType()))
return false;
if (!canBeSafelyCasted(from_type_map.getValueType(), to_type_map.getValueType()))
return false;
return true;
}
if (to_which_type.isArray())
{
// Map nested type is Array(Tuple(key_type, value_type))
const auto & from_type_map = assert_cast<const DataTypeMap &>(*from_type);
const auto & to_type_array = assert_cast<const DataTypeArray &>(*to_type_unwrapped);
const auto * to_type_nested_tuple_type = typeid_cast<const DataTypeTuple *>(to_type_array.getNestedType().get());
if (!to_type_nested_tuple_type)
return false;
const auto & to_type_tuple_elements = to_type_nested_tuple_type->getElements();
if (to_type_tuple_elements.size() != 2)
return false;
if (!canBeSafelyCasted(from_type_map.getKeyType(), to_type_tuple_elements[0]))
return false;
if (!canBeSafelyCasted(from_type_map.getValueType(), to_type_tuple_elements[1]))
return false;
return true;
}
if (to_which_type.isString())
return true;
return false;
}
case TypeIndex::Tuple:
{
if (to_which_type.isTuple())
{
const auto & from_type_tuple = assert_cast<const DataTypeTuple &>(*from_type);
const auto & to_type_tuple = assert_cast<const DataTypeTuple &>(*to_type_unwrapped);
const auto & from_tuple_type_elements = from_type_tuple.getElements();
const auto & to_tuple_type_elements = to_type_tuple.getElements();
size_t lhs_type_elements_size = from_tuple_type_elements.size();
if (lhs_type_elements_size != to_tuple_type_elements.size())
return false;
for (size_t i = 0; i < lhs_type_elements_size; ++i)
if (!canBeSafelyCasted(from_tuple_type_elements[i], to_tuple_type_elements[i]))
return false;
return true;
}
if (to_which_type.isString())
return true;
return false;
}
case TypeIndex::String:
case TypeIndex::Object:
case TypeIndex::Set:
case TypeIndex::Interval:
case TypeIndex::Function:
case TypeIndex::AggregateFunction:
case TypeIndex::Nothing:
return false;
}
return true;
}
}

19
src/DataTypes/Utils.h Normal file
View File

@ -0,0 +1,19 @@
#pragma once
#include <DataTypes/IDataType.h>
namespace DB
{
/** Returns true if from_type can be safely casted to to_type.
*
* Examples:
* From type UInt8 to type UInt16 returns true.
* From type UInt16 to type UInt8 returns false.
* From type String to type LowCardinality(String) returns true.
* From type LowCardinality(String) to type String returns true.
* From type String to type UInt8 returns false.
*/
bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_type);
}

View File

@ -324,7 +324,7 @@ ReservationPtr DiskEncrypted::reserve(UInt64 bytes)
}
void DiskEncrypted::copyDirectoryContent(const String & from_dir, const std::shared_ptr<IDisk> & to_disk, const String & to_dir, const WriteSettings & settings)
void DiskEncrypted::copyDirectoryContent(const String & from_dir, const std::shared_ptr<IDisk> & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings)
{
/// Check if we can copy the file without deciphering.
if (isSameDiskType(*this, *to_disk))
@ -340,14 +340,14 @@ void DiskEncrypted::copyDirectoryContent(const String & from_dir, const std::sha
auto wrapped_from_path = wrappedPath(from_dir);
auto to_delegate = to_disk_enc->delegate;
auto wrapped_to_path = to_disk_enc->wrappedPath(to_dir);
delegate->copyDirectoryContent(wrapped_from_path, to_delegate, wrapped_to_path, settings);
delegate->copyDirectoryContent(wrapped_from_path, to_delegate, wrapped_to_path, read_settings, write_settings);
return;
}
}
}
/// Copy the file through buffers with deciphering.
IDisk::copyDirectoryContent(from_dir, to_disk, to_dir, settings);
IDisk::copyDirectoryContent(from_dir, to_disk, to_dir, read_settings, write_settings);
}
std::unique_ptr<ReadBufferFromFileBase> DiskEncrypted::readFile(

View File

@ -112,7 +112,7 @@ public:
delegate->listFiles(wrapped_path, file_names);
}
void copyDirectoryContent(const String & from_dir, const std::shared_ptr<IDisk> & to_disk, const String & to_dir, const WriteSettings & settings) override;
void copyDirectoryContent(const String & from_dir, const std::shared_ptr<IDisk> & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings) override;
std::unique_ptr<ReadBufferFromFileBase> readFile(
const String & path,

View File

@ -53,11 +53,11 @@ String DiskEncryptedSettings::findKeyByFingerprint(UInt128 key_fingerprint, cons
return it->second;
}
void DiskEncryptedTransaction::copyFile(const std::string & from_file_path, const std::string & to_file_path, const WriteSettings & settings)
void DiskEncryptedTransaction::copyFile(const std::string & from_file_path, const std::string & to_file_path, const ReadSettings & read_settings, const WriteSettings & write_settings)
{
auto wrapped_from_path = wrappedPath(from_file_path);
auto wrapped_to_path = wrappedPath(to_file_path);
delegate_transaction->copyFile(wrapped_from_path, wrapped_to_path, settings);
delegate_transaction->copyFile(wrapped_from_path, wrapped_to_path, read_settings, write_settings);
}
std::unique_ptr<WriteBufferFromFileBase> DiskEncryptedTransaction::writeFile( // NOLINT

View File

@ -116,7 +116,7 @@ public:
/// but it's impossible to implement correctly in transactions because other disk can
/// use different metadata storage.
/// TODO: maybe remove it at all, we don't want copies
void copyFile(const std::string & from_file_path, const std::string & to_file_path, const WriteSettings & settings) override;
void copyFile(const std::string & from_file_path, const std::string & to_file_path, const ReadSettings & read_settings, const WriteSettings & write_settings) override;
/// Open the file for write and return WriteBufferFromFileBase object.
std::unique_ptr<WriteBufferFromFileBase> writeFile( /// NOLINT

View File

@ -432,13 +432,13 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another)
return typeid(one) == typeid(another);
}
void DiskLocal::copyDirectoryContent(const String & from_dir, const std::shared_ptr<IDisk> & to_disk, const String & to_dir, const WriteSettings & settings)
void DiskLocal::copyDirectoryContent(const String & from_dir, const std::shared_ptr<IDisk> & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings)
{
/// If throttling was configured we cannot use copying directly.
if (isSameDiskType(*this, *to_disk) && !settings.local_throttler)
if (isSameDiskType(*this, *to_disk) && !read_settings.local_throttler && !write_settings.local_throttler)
fs::copy(fs::path(disk_path) / from_dir, fs::path(to_disk->getPath()) / to_dir, fs::copy_options::recursive | fs::copy_options::overwrite_existing); /// Use more optimal way.
else
IDisk::copyDirectoryContent(from_dir, to_disk, to_dir, settings);
IDisk::copyDirectoryContent(from_dir, to_disk, to_dir, read_settings, write_settings);
}
SyncGuardPtr DiskLocal::getDirectorySyncGuard(const String & path) const

View File

@ -65,7 +65,7 @@ public:
void replaceFile(const String & from_path, const String & to_path) override;
void copyDirectoryContent(const String & from_dir, const std::shared_ptr<IDisk> & to_disk, const String & to_dir, const WriteSettings & settings) override;
void copyDirectoryContent(const String & from_dir, const std::shared_ptr<IDisk> & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings) override;
void listFiles(const String & path, std::vector<String> & file_names) const override;

View File

@ -54,9 +54,9 @@ public:
disk.replaceFile(from_path, to_path);
}
void copyFile(const std::string & from_file_path, const std::string & to_file_path, const WriteSettings & settings) override
void copyFile(const std::string & from_file_path, const std::string & to_file_path, const ReadSettings & read_settings, const WriteSettings & write_settings) override
{
disk.copyFile(from_file_path, disk, to_file_path, settings);
disk.copyFile(from_file_path, disk, to_file_path, read_settings, write_settings);
}
std::unique_ptr<WriteBufferFromFileBase> writeFile( /// NOLINT

View File

@ -24,13 +24,13 @@ bool IDisk::isDirectoryEmpty(const String & path) const
return !iterateDirectory(path)->isValid();
}
void IDisk::copyFile(const String & from_file_path, IDisk & to_disk, const String & to_file_path, const WriteSettings & settings) /// NOLINT
void IDisk::copyFile(const String & from_file_path, IDisk & to_disk, const String & to_file_path, const ReadSettings & read_settings, const WriteSettings & write_settings) /// NOLINT
{
LOG_DEBUG(&Poco::Logger::get("IDisk"), "Copying from {} (path: {}) {} to {} (path: {}) {}.",
getName(), getPath(), from_file_path, to_disk.getName(), to_disk.getPath(), to_file_path);
auto in = readFile(from_file_path);
auto out = to_disk.writeFile(to_file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, settings);
auto in = readFile(from_file_path, read_settings);
auto out = to_disk.writeFile(to_file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, write_settings);
copyData(*in, *out);
out->finalize();
}
@ -80,7 +80,7 @@ UInt128 IDisk::getEncryptedFileIV(const String &) const
using ResultsCollector = std::vector<std::future<void>>;
void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_path, ThreadPool & pool, ResultsCollector & results, bool copy_root_dir, const WriteSettings & settings)
void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_path, ThreadPool & pool, ResultsCollector & results, bool copy_root_dir, const ReadSettings & read_settings, const WriteSettings & write_settings)
{
if (from_disk.isFile(from_path))
{
@ -88,7 +88,7 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p
auto future = promise->get_future();
pool.scheduleOrThrowOnError(
[&from_disk, from_path, &to_disk, to_path, &settings, promise, thread_group = CurrentThread::getGroup()]()
[&from_disk, from_path, &to_disk, to_path, &read_settings, &write_settings, promise, thread_group = CurrentThread::getGroup()]()
{
try
{
@ -97,7 +97,7 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p
if (thread_group)
CurrentThread::attachToGroup(thread_group);
from_disk.copyFile(from_path, to_disk, fs::path(to_path) / fileName(from_path), settings);
from_disk.copyFile(from_path, to_disk, fs::path(to_path) / fileName(from_path), read_settings, write_settings);
promise->set_value();
}
catch (...)
@ -119,19 +119,19 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p
}
for (auto it = from_disk.iterateDirectory(from_path); it->isValid(); it->next())
asyncCopy(from_disk, it->path(), to_disk, dest, pool, results, true, settings);
asyncCopy(from_disk, it->path(), to_disk, dest, pool, results, true, read_settings, write_settings);
}
}
void IDisk::copyThroughBuffers(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path, bool copy_root_dir, WriteSettings settings)
void IDisk::copyThroughBuffers(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path, bool copy_root_dir, const ReadSettings & read_settings, WriteSettings write_settings)
{
ResultsCollector results;
/// Disable parallel write. We already copy in parallel.
/// Avoid high memory usage. See test_s3_zero_copy_ttl/test.py::test_move_and_s3_memory_usage
settings.s3_allow_parallel_part_upload = false;
write_settings.s3_allow_parallel_part_upload = false;
asyncCopy(*this, from_path, *to_disk, to_path, copying_thread_pool, results, copy_root_dir, settings);
asyncCopy(*this, from_path, *to_disk, to_path, copying_thread_pool, results, copy_root_dir, read_settings, write_settings);
for (auto & result : results)
result.wait();
@ -140,12 +140,12 @@ void IDisk::copyThroughBuffers(const String & from_path, const std::shared_ptr<I
}
void IDisk::copyDirectoryContent(const String & from_dir, const std::shared_ptr<IDisk> & to_disk, const String & to_dir, const WriteSettings & settings)
void IDisk::copyDirectoryContent(const String & from_dir, const std::shared_ptr<IDisk> & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings)
{
if (!to_disk->exists(to_dir))
to_disk->createDirectories(to_dir);
copyThroughBuffers(from_dir, to_disk, to_dir, /* copy_root_dir= */ false, settings);
copyThroughBuffers(from_dir, to_disk, to_dir, /* copy_root_dir= */ false, read_settings, write_settings);
}
void IDisk::truncateFile(const String &, size_t)

View File

@ -193,14 +193,15 @@ public:
virtual void replaceFile(const String & from_path, const String & to_path) = 0;
/// Recursively copy files from from_dir to to_dir. Create to_dir if not exists.
virtual void copyDirectoryContent(const String & from_dir, const std::shared_ptr<IDisk> & to_disk, const String & to_dir, const WriteSettings & settings);
virtual void copyDirectoryContent(const String & from_dir, const std::shared_ptr<IDisk> & to_disk, const String & to_dir, const ReadSettings & read_settings, const WriteSettings & write_settings);
/// Copy file `from_file_path` to `to_file_path` located at `to_disk`.
virtual void copyFile( /// NOLINT
const String & from_file_path,
IDisk & to_disk,
const String & to_file_path,
const WriteSettings & settings = {});
const ReadSettings & read_settings = {},
const WriteSettings & write_settings = {});
/// List files at `path` and add their names to `file_names`
virtual void listFiles(const String & path, std::vector<String> & file_names) const = 0;
@ -470,7 +471,7 @@ protected:
/// Base implementation of the function copy().
/// It just opens two files, reads data by portions from the first file, and writes it to the second one.
/// A derived class may override copy() to provide a faster implementation.
void copyThroughBuffers(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path, bool copy_root_dir, WriteSettings settings);
void copyThroughBuffers(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path, bool copy_root_dir, const ReadSettings & read_settings, WriteSettings write_settings);
virtual void checkAccessImpl(const String & path);

View File

@ -59,7 +59,11 @@ public:
/// but it's impossible to implement correctly in transactions because other disk can
/// use different metadata storage.
/// TODO: maybe remove it at all, we don't want copies
virtual void copyFile(const std::string & from_file_path, const std::string & to_file_path, const WriteSettings & settings = {}) = 0;
virtual void copyFile(
const std::string & from_file_path,
const std::string & to_file_path,
const ReadSettings & read_settings = {},
const WriteSettings & write_settings = {}) = 0;
/// Open the file for write and return WriteBufferFromFileBase object.
virtual std::unique_ptr<WriteBufferFromFileBase> writeFile( /// NOLINT

View File

@ -357,6 +357,8 @@ ObjectMetadata AzureObjectStorage::getObjectMetadata(const std::string & path) c
void AzureObjectStorage::copyObject( /// NOLINT
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings &,
const WriteSettings &,
std::optional<ObjectAttributes> object_to_attributes)
{
auto client_ptr = client.get();

View File

@ -100,6 +100,8 @@ public:
void copyObject( /// NOLINT
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
std::optional<ObjectAttributes> object_to_attributes = {}) override;
void shutdown() override {}

View File

@ -160,16 +160,22 @@ void CachedObjectStorage::removeObjectsIfExist(const StoredObjects & objects)
void CachedObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
IObjectStorage & object_storage_to,
std::optional<ObjectAttributes> object_to_attributes)
{
object_storage->copyObjectToAnotherObjectStorage(object_from, object_to, object_storage_to, object_to_attributes);
object_storage->copyObjectToAnotherObjectStorage(object_from, object_to, read_settings, write_settings, object_storage_to, object_to_attributes);
}
void CachedObjectStorage::copyObject( // NOLINT
const StoredObject & object_from, const StoredObject & object_to, std::optional<ObjectAttributes> object_to_attributes)
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
std::optional<ObjectAttributes> object_to_attributes)
{
object_storage->copyObject(object_from, object_to, object_to_attributes);
object_storage->copyObject(object_from, object_to, read_settings, write_settings, object_to_attributes);
}
std::unique_ptr<IObjectStorage> CachedObjectStorage::cloneObjectStorage(

View File

@ -57,11 +57,15 @@ public:
void copyObject( /// NOLINT
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
std::optional<ObjectAttributes> object_to_attributes = {}) override;
void copyObjectToAnotherObjectStorage( /// NOLINT
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
IObjectStorage & object_storage_to,
std::optional<ObjectAttributes> object_to_attributes = {}) override;

View File

@ -68,7 +68,7 @@ DiskObjectStorage::DiskObjectStorage(
, send_metadata(config.getBool(config_prefix + ".send_metadata", false))
, read_resource_name(config.getString(config_prefix + ".read_resource", ""))
, write_resource_name(config.getString(config_prefix + ".write_resource", ""))
, metadata_helper(std::make_unique<DiskObjectStorageRemoteMetadataRestoreHelper>(this, ReadSettings{}))
, metadata_helper(std::make_unique<DiskObjectStorageRemoteMetadataRestoreHelper>(this, ReadSettings{}, WriteSettings{}))
{}
StoredObjects DiskObjectStorage::getStorageObjects(const String & local_path) const
@ -180,7 +180,8 @@ void DiskObjectStorage::copyFile( /// NOLINT
const String & from_file_path,
IDisk & to_disk,
const String & to_file_path,
const WriteSettings & settings)
const ReadSettings & read_settings,
const WriteSettings & write_settings)
{
if (this == &to_disk)
{
@ -192,7 +193,7 @@ void DiskObjectStorage::copyFile( /// NOLINT
else
{
/// Copy through buffers
IDisk::copyFile(from_file_path, to_disk, to_file_path, settings);
IDisk::copyFile(from_file_path, to_disk, to_file_path, read_settings, write_settings);
}
}

View File

@ -162,7 +162,8 @@ public:
const String & from_file_path,
IDisk & to_disk,
const String & to_file_path,
const WriteSettings & settings = {}) override;
const ReadSettings & read_settings = {},
const WriteSettings & write_settings = {}) override;
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context_, const String &, const DisksMap &) override;

View File

@ -84,7 +84,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::saveSchemaVersion(const int &
{
StoredObject object{fs::path(disk->object_storage_root_path) / SCHEMA_VERSION_OBJECT};
auto buf = disk->object_storage->writeObject(object, WriteMode::Rewrite);
auto buf = disk->object_storage->writeObject(object, WriteMode::Rewrite, /* attributes= */ {}, /* buf_size= */ DBMS_DEFAULT_BUFFER_SIZE, write_settings);
writeIntText(version, *buf);
buf->finalize();
@ -93,7 +93,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::saveSchemaVersion(const int &
void DiskObjectStorageRemoteMetadataRestoreHelper::updateObjectMetadata(const String & key, const ObjectAttributes & metadata) const
{
StoredObject object{key};
disk->object_storage->copyObject(object, object, metadata);
disk->object_storage->copyObject(object, object, read_settings, write_settings, metadata);
}
void DiskObjectStorageRemoteMetadataRestoreHelper::migrateFileToRestorableSchema(const String & path) const
@ -434,7 +434,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::processRestoreFiles(
/// Copy object if we restore to different bucket / path.
if (source_object_storage->getObjectsNamespace() != disk->object_storage->getObjectsNamespace() || disk->object_storage_root_path != source_path)
source_object_storage->copyObjectToAnotherObjectStorage(object_from, object_to, *disk->object_storage);
source_object_storage->copyObjectToAnotherObjectStorage(object_from, object_to, read_settings, write_settings, *disk->object_storage);
auto tx = disk->metadata_storage->createTransaction();
tx->addBlobToMetadata(path, relative_key, meta.size_bytes);

View File

@ -24,9 +24,10 @@ public:
static constexpr UInt64 LATEST_REVISION = std::numeric_limits<UInt64>::max();
static constexpr UInt64 UNKNOWN_REVISION = 0;
DiskObjectStorageRemoteMetadataRestoreHelper(DiskObjectStorage * disk_, ReadSettings read_settings_)
DiskObjectStorageRemoteMetadataRestoreHelper(DiskObjectStorage * disk_, ReadSettings read_settings_, WriteSettings write_settings_)
: disk(disk_)
, read_settings(std::move(read_settings_))
, write_settings(std::move(write_settings_))
, operation_log_suffix("-" + getFQDNOrHostName())
{
}
@ -94,6 +95,7 @@ private:
ObjectStoragePtr object_storage_from_another_namespace;
ReadSettings read_settings;
WriteSettings write_settings;
String operation_log_suffix;
};

View File

@ -1,6 +1,7 @@
#include <Disks/ObjectStorages/DiskObjectStorageTransaction.h>
#include <Disks/ObjectStorages/DiskObjectStorage.h>
#include <Disks/IO/WriteBufferWithFinalizeCallback.h>
#include <Interpreters/Context.h>
#include <Common/checkStackSize.h>
#include <ranges>
#include <Common/logger_useful.h>
@ -474,6 +475,9 @@ struct WriteFileObjectStorageOperation final : public IDiskObjectStorageOperatio
struct CopyFileObjectStorageOperation final : public IDiskObjectStorageOperation
{
ReadSettings read_settings;
WriteSettings write_settings;
/// Local paths
std::string from_path;
std::string to_path;
@ -483,9 +487,13 @@ struct CopyFileObjectStorageOperation final : public IDiskObjectStorageOperation
CopyFileObjectStorageOperation(
IObjectStorage & object_storage_,
IMetadataStorage & metadata_storage_,
const ReadSettings & read_settings_,
const WriteSettings & write_settings_,
const std::string & from_path_,
const std::string & to_path_)
: IDiskObjectStorageOperation(object_storage_, metadata_storage_)
, read_settings(read_settings_)
, write_settings(write_settings_)
, from_path(from_path_)
, to_path(to_path_)
{}
@ -505,7 +513,7 @@ struct CopyFileObjectStorageOperation final : public IDiskObjectStorageOperation
std::string blob_name = object_storage.generateBlobNameForPath(to_path);
auto object_to = StoredObject(fs::path(metadata_storage.getObjectStorageRootPath()) / blob_name);
object_storage.copyObject(object_from, object_to);
object_storage.copyObject(object_from, object_to, read_settings, write_settings);
tx->addBlobToMetadata(to_path, blob_name, object_from.bytes_size);
@ -810,13 +818,10 @@ void DiskObjectStorageTransaction::createFile(const std::string & path)
}));
}
void DiskObjectStorageTransaction::copyFile(const std::string & from_file_path, const std::string & to_file_path, const WriteSettings & settings)
void DiskObjectStorageTransaction::copyFile(const std::string & from_file_path, const std::string & to_file_path, const ReadSettings & read_settings, const WriteSettings & write_settings)
{
/// NOTE: For native copy we can ignore throttling, so no need to use WriteSettings
UNUSED(settings);
operations_to_execute.emplace_back(
std::make_unique<CopyFileObjectStorageOperation>(object_storage, metadata_storage, from_file_path, to_file_path));
std::make_unique<CopyFileObjectStorageOperation>(object_storage, metadata_storage, read_settings, write_settings, from_file_path, to_file_path));
}
void DiskObjectStorageTransaction::commit()

View File

@ -86,7 +86,7 @@ public:
void createFile(const String & path) override;
void copyFile(const std::string & from_file_path, const std::string & to_file_path, const WriteSettings & settings) override;
void copyFile(const std::string & from_file_path, const std::string & to_file_path, const ReadSettings & read_settings, const WriteSettings &) override;
/// writeFile is a difficult function for transactions.
/// Now it's almost noop because metadata added to transaction in finalize method

View File

@ -133,6 +133,8 @@ ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string &) const
void HDFSObjectStorage::copyObject( /// NOLINT
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
std::optional<ObjectAttributes> object_to_attributes)
{
if (object_to_attributes.has_value())
@ -140,8 +142,8 @@ void HDFSObjectStorage::copyObject( /// NOLINT
ErrorCodes::UNSUPPORTED_METHOD,
"HDFS API doesn't support custom attributes/metadata for stored objects");
auto in = readObject(object_from);
auto out = writeObject(object_to, WriteMode::Rewrite);
auto in = readObject(object_from, read_settings);
auto out = writeObject(object_to, WriteMode::Rewrite, /* attributes= */ {}, /* buf_size= */ DBMS_DEFAULT_BUFFER_SIZE, write_settings);
copyData(*in, *out);
out->finalize();
}

View File

@ -98,6 +98,8 @@ public:
void copyObject( /// NOLINT
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
std::optional<ObjectAttributes> object_to_attributes = {}) override;
void shutdown() override;

View File

@ -62,14 +62,16 @@ ThreadPool & IObjectStorage::getThreadPoolWriter()
void IObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
IObjectStorage & object_storage_to,
std::optional<ObjectAttributes> object_to_attributes)
{
if (&object_storage_to == this)
copyObject(object_from, object_to, object_to_attributes);
copyObject(object_from, object_to, read_settings, write_settings, object_to_attributes);
auto in = readObject(object_from);
auto out = object_storage_to.writeObject(object_to, WriteMode::Rewrite);
auto in = readObject(object_from, read_settings);
auto out = object_storage_to.writeObject(object_to, WriteMode::Rewrite, /* attributes= */ {}, /* buf_size= */ DBMS_DEFAULT_BUFFER_SIZE, write_settings);
copyData(*in, *out);
out->finalize();
}

View File

@ -131,6 +131,8 @@ public:
virtual void copyObject( /// NOLINT
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
std::optional<ObjectAttributes> object_to_attributes = {}) = 0;
/// Copy object to another instance of object storage
@ -139,6 +141,8 @@ public:
virtual void copyObjectToAnotherObjectStorage( /// NOLINT
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
IObjectStorage & object_storage_to,
std::optional<ObjectAttributes> object_to_attributes = {});

View File

@ -167,10 +167,14 @@ 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 */)
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
std::optional<ObjectAttributes> /* object_to_attributes */)
{
auto in = readObject(object_from);
auto out = writeObject(object_to, WriteMode::Rewrite);
auto in = readObject(object_from, read_settings);
auto out = writeObject(object_to, WriteMode::Rewrite, /* attributes= */ {}, /* buf_size= */ DBMS_DEFAULT_BUFFER_SIZE, write_settings);
copyData(*in, *out);
out->finalize();
}

View File

@ -57,6 +57,8 @@ public:
void copyObject( /// NOLINT
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
std::optional<ObjectAttributes> object_to_attributes = {}) override;
void shutdown() override;

View File

@ -425,6 +425,8 @@ ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) cons
void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
IObjectStorage & object_storage_to,
std::optional<ObjectAttributes> object_to_attributes)
{
@ -435,24 +437,48 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT
auto settings_ptr = s3_settings.get();
auto size = S3::getObjectSize(*clients_->client, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true);
auto scheduler = threadPoolCallbackRunner<void>(getThreadPoolWriter(), "S3ObjStor_copy");
copyS3File(clients_->client, clients_->client_with_long_timeout, 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);
copyS3File(clients_->client,
clients_->client_with_long_timeout,
bucket,
object_from.remote_path,
0,
size,
dest_s3->bucket,
object_to.remote_path,
settings_ptr->request_settings,
patchSettings(read_settings),
object_to_attributes,
scheduler,
/* for_disk_s3= */ true);
}
else
{
IObjectStorage::copyObjectToAnotherObjectStorage(object_from, object_to, object_storage_to, object_to_attributes);
}
IObjectStorage::copyObjectToAnotherObjectStorage(object_from, object_to, read_settings, write_settings, object_storage_to, object_to_attributes);
}
void S3ObjectStorage::copyObject( // NOLINT
const StoredObject & object_from, const StoredObject & object_to, std::optional<ObjectAttributes> object_to_attributes)
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings &,
std::optional<ObjectAttributes> object_to_attributes)
{
auto clients_ = clients.get();
auto settings_ptr = s3_settings.get();
auto size = S3::getObjectSize(*clients_->client, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true);
auto scheduler = threadPoolCallbackRunner<void>(getThreadPoolWriter(), "S3ObjStor_copy");
copyS3File(clients_->client, clients_->client_with_long_timeout, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path,
settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true);
copyS3File(clients_->client,
clients_->client_with_long_timeout,
bucket,
object_from.remote_path,
0,
size,
bucket,
object_to.remote_path,
settings_ptr->request_settings,
patchSettings(read_settings),
object_to_attributes,
scheduler,
/* for_disk_s3= */ true);
}
void S3ObjectStorage::setNewSettings(std::unique_ptr<S3ObjectStorageSettings> && s3_settings_)

View File

@ -135,11 +135,15 @@ public:
void copyObject( /// NOLINT
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
std::optional<ObjectAttributes> object_to_attributes = {}) override;
void copyObjectToAnotherObjectStorage( /// NOLINT
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
IObjectStorage & object_storage_to,
std::optional<ObjectAttributes> object_to_attributes = {}) override;

View File

@ -264,7 +264,7 @@ void WebObjectStorage::removeObjectsIfExist(const StoredObjects &)
throwNotAllowed();
}
void WebObjectStorage::copyObject(const StoredObject &, const StoredObject &, std::optional<ObjectAttributes>) // NOLINT
void WebObjectStorage::copyObject(const StoredObject &, const StoredObject &, const ReadSettings &, const WriteSettings &, std::optional<ObjectAttributes>) // NOLINT
{
throwNotAllowed();
}

View File

@ -68,6 +68,8 @@ public:
void copyObject( /// NOLINT
const StoredObject & object_from,
const StoredObject & object_to,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
std::optional<ObjectAttributes> object_to_attributes = {}) override;
void shutdown() override;

View File

@ -6,7 +6,6 @@
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnLowCardinality.h>
#include <Common/assert_cast.h>
#include <DataTypes/DataTypeNullable.h>
namespace DB

View File

@ -199,6 +199,7 @@ public:
/// Parse JSON for every row
Impl impl;
GeneratorJSONPath<JSONParser> generator_json_path(res);
for (const auto i : collections::range(0, input_rows_count))
{
std::string_view json{
@ -208,7 +209,9 @@ public:
bool added_to_column = false;
if (document_ok)
{
added_to_column = impl.insertResultToColumn(*to, document, res, context);
/// Instead of creating a new generator for each row, we can reuse the same one.
generator_json_path.reinitialize();
added_to_column = impl.insertResultToColumn(*to, document, generator_json_path, context);
}
if (!added_to_column)
{
@ -287,9 +290,8 @@ public:
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const ContextPtr &)
static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath<JSONParser> & generator_json_path, const ContextPtr &)
{
GeneratorJSONPath<JSONParser> generator_json_path(query_ptr);
Element current_element = root;
VisitorStatus status;
while ((status = generator_json_path.getNextItem(current_element)) != VisitorStatus::Exhausted)
@ -337,9 +339,8 @@ public:
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const ContextPtr & context)
static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath<JSONParser> & generator_json_path, const ContextPtr & context)
{
GeneratorJSONPath<JSONParser> generator_json_path(query_ptr);
Element current_element = root;
VisitorStatus status;
@ -405,11 +406,10 @@ public:
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
static bool insertResultToColumn(IColumn & dest, const Element & root, ASTPtr & query_ptr, const ContextPtr &)
static bool insertResultToColumn(IColumn & dest, const Element & root, GeneratorJSONPath<JSONParser> & generator_json_path, const ContextPtr &)
{
ColumnString & col_str = assert_cast<ColumnString &>(dest);
GeneratorJSONPath<JSONParser> generator_json_path(query_ptr);
Element current_element = root;
VisitorStatus status;
bool success = false;

View File

@ -18,6 +18,10 @@
#include "vec_crc32.h"
#endif
#if defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__
#include <crc32-s390x.h>
#endif
namespace DB
{
@ -43,7 +47,7 @@ struct Hash
#elif (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
return crc32_ppc(crc, reinterpret_cast<const unsigned char *>(&val), sizeof(val));
#elif defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__
return s390x_crc32(crc, val);
return crc32c_le(static_cast<UInt32>(crc), reinterpret_cast<unsigned char *>(&val), sizeof(val));
#else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "String hash is not implemented without sse4.2 support");
#endif
@ -58,7 +62,7 @@ struct Hash
#elif (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
return crc32_ppc(crc, reinterpret_cast<const unsigned char *>(&val), sizeof(val));
#elif defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__
return s390x_crc32_u32(crc, val);
return crc32c_le(static_cast<UInt32>(crc), reinterpret_cast<unsigned char *>(&val), sizeof(val));
#else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "String hash is not implemented without sse4.2 support");
#endif
@ -73,7 +77,7 @@ struct Hash
#elif (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
return crc32_ppc(crc, reinterpret_cast<const unsigned char *>(&val), sizeof(val));
#elif defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__
return s390x_crc32_u16(crc, val);
return crc32c_le(static_cast<UInt32>(crc), reinterpret_cast<unsigned char *>(&val), sizeof(val));
#else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "String hash is not implemented without sse4.2 support");
#endif
@ -88,7 +92,7 @@ struct Hash
#elif (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
return crc32_ppc(crc, reinterpret_cast<const unsigned char *>(&val), sizeof(val));
#elif defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__
return s390x_crc32_u8(crc, val);
return crc32c_le(static_cast<UInt32>(crc), reinterpret_cast<unsigned char *>(&val), sizeof(val));
#else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "String hash is not implemented without sse4.2 support");
#endif

View File

@ -105,6 +105,16 @@ public:
}
}
void reinitialize()
{
while (current_visitor >= 0)
{
visitors[current_visitor]->reinitialize();
current_visitor--;
}
current_visitor = 0;
}
private:
bool updateVisitorsForNextRun()
{

View File

@ -52,6 +52,15 @@ public:
{
return arguments[0].column->convertToFullColumnIfConst();
}
bool hasInformationAboutMonotonicity() const override { return true; }
Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override
{
/// Depending on the argument the function materialize() is either a constant or works as identity().
/// In both cases this function is monotonic and non-decreasing.
return {.is_monotonic = true, .is_always_monotonic = true};
}
};
}

View File

@ -321,7 +321,7 @@ namespace
/// To avoid such a deadlock we unlock `lock` before entering `pool_ptr->second->get`.
lock.unlock();
auto retry_timeout = timeouts.connection_timeout.totalMicroseconds();
auto retry_timeout = timeouts.connection_timeout.totalMilliseconds();
auto session = pool_ptr->second->get(retry_timeout);
setTimeouts(*session, timeouts);

View File

@ -610,6 +610,7 @@ namespace
const String & dest_bucket_,
const String & dest_key_,
const S3Settings::RequestSettings & request_settings_,
const ReadSettings & read_settings_,
const std::optional<std::map<String, String>> & object_metadata_,
ThreadPoolCallbackRunner<void> schedule_,
bool for_disk_s3_)
@ -619,6 +620,7 @@ namespace
, offset(src_offset_)
, size(src_size_)
, supports_multipart_copy(client_ptr_->supportsMultiPartCopy())
, read_settings(read_settings_)
{
}
@ -639,12 +641,13 @@ namespace
size_t offset;
size_t size;
bool supports_multipart_copy;
const ReadSettings read_settings;
CreateReadBuffer getSourceObjectReadBuffer()
{
return [&]
{
return std::make_unique<ReadBufferFromS3>(client_ptr, src_bucket, src_key, "", request_settings, Context::getGlobalContextInstance()->getReadSettings());
return std::make_unique<ReadBufferFromS3>(client_ptr, src_bucket, src_key, "", request_settings, read_settings);
};
}
@ -826,20 +829,21 @@ void copyS3File(
const String & dest_bucket,
const String & dest_key,
const S3Settings::RequestSettings & settings,
const ReadSettings & read_settings,
const std::optional<std::map<String, String>> & object_metadata,
ThreadPoolCallbackRunner<void> schedule,
bool for_disk_s3)
{
if (settings.allow_native_copy)
{
CopyFileHelper helper{s3_client, s3_client_with_long_timeout, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3};
CopyFileHelper helper{s3_client, s3_client_with_long_timeout, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, read_settings, object_metadata, schedule, for_disk_s3};
helper.performCopy();
}
else
{
auto create_read_buffer = [&]
{
return std::make_unique<ReadBufferFromS3>(s3_client, src_bucket, src_key, "", settings, Context::getGlobalContextInstance()->getReadSettings());
return std::make_unique<ReadBufferFromS3>(s3_client, src_bucket, src_key, "", settings, read_settings);
};
copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, s3_client_with_long_timeout, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3);
}

View File

@ -31,6 +31,8 @@ using CreateReadBuffer = std::function<std::unique_ptr<SeekableReadBuffer>()>;
/// CompleteMultipartUpload requests. These requests need longer timeout because S3 servers often
/// block on them for multiple seconds without sending or receiving data from us (maybe the servers
/// are copying data internally, or maybe throttling, idk).
///
/// read_settings - is used for throttling in case of native copy is not possible
void copyS3File(
const std::shared_ptr<const S3::Client> & s3_client,
const std::shared_ptr<const S3::Client> & s3_client_with_long_timeout,
@ -41,6 +43,7 @@ void copyS3File(
const String & dest_bucket,
const String & dest_key,
const S3Settings::RequestSettings & settings,
const ReadSettings & read_settings,
const std::optional<std::map<String, String>> & object_metadata = std::nullopt,
ThreadPoolCallbackRunner<void> schedule_ = {},
bool for_disk_s3 = false);

View File

@ -34,7 +34,7 @@ namespace ErrorCodes
namespace ClusterProxy
{
ContextMutablePtr updateSettingsForCluster(bool interserver_mode,
ContextMutablePtr updateSettingsForCluster(const Cluster & cluster,
ContextPtr context,
const Settings & settings,
const StorageID & main_table,
@ -47,6 +47,7 @@ ContextMutablePtr updateSettingsForCluster(bool interserver_mode,
/// If "secret" (in remote_servers) is not in use,
/// user on the shard is not the same as the user on the initiator,
/// hence per-user limits should not be applied.
const bool interserver_mode = !cluster.getSecret().empty();
if (!interserver_mode)
{
/// Does not matter on remote servers, because queries are sent under different user.
@ -123,6 +124,22 @@ ContextMutablePtr updateSettingsForCluster(bool interserver_mode,
new_settings.additional_table_filters.value.push_back(std::move(tuple));
}
/// disable parallel replicas if cluster contains only shards with 1 replica
if (context->canUseParallelReplicas())
{
bool disable_parallel_replicas = true;
for (const auto & shard : cluster.getShardsInfo())
{
if (shard.getAllNodeCount() > 1)
{
disable_parallel_replicas = false;
break;
}
}
if (disable_parallel_replicas)
new_settings.allow_experimental_parallel_reading_from_replicas = false;
}
auto new_context = Context::createCopy(context);
new_context->setSettings(new_settings);
return new_context;
@ -174,11 +191,20 @@ void executeQuery(
std::vector<QueryPlanPtr> plans;
SelectStreamFactory::Shards remote_shards;
auto new_context = updateSettingsForCluster(!not_optimized_cluster->getSecret().empty(), context, settings,
main_table, query_info.additional_filter_ast, log);
auto cluster = query_info.getCluster();
auto new_context = updateSettingsForCluster(*cluster, context, settings, main_table, query_info.additional_filter_ast, log);
if (context->getSettingsRef().allow_experimental_parallel_reading_from_replicas
&& context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value
!= new_context->getSettingsRef().allow_experimental_parallel_reading_from_replicas.value)
{
LOG_TRACE(
log,
"Parallel reading from replicas is disabled for cluster. There are no shards with more than 1 replica: cluster={}",
cluster->getName());
}
new_context->increaseDistributedDepth();
ClusterPtr cluster = query_info.getCluster();
const size_t shards = cluster->getShardCount();
for (size_t i = 0, s = cluster->getShardsInfo().size(); i < s; ++i)
{

View File

@ -37,7 +37,7 @@ class SelectStreamFactory;
/// - optimize_skip_unused_shards_nesting
///
/// @return new Context with adjusted settings
ContextMutablePtr updateSettingsForCluster(bool interserver_mode,
ContextMutablePtr updateSettingsForCluster(const Cluster & cluster,
ContextPtr context,
const Settings & settings,
const StorageID & main_table,

View File

@ -16,6 +16,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int INVALID_IDENTIFIER;
}
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableIdentifier & identifier, const String & current_database)
@ -37,7 +38,7 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & ident
else if (identifier.name_parts.size() == 1)
table = identifier.name_parts[0];
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: invalid identifier");
throw Exception(ErrorCodes::INVALID_IDENTIFIER, "Invalid identifier");
if (database.empty())
database = current_database;
@ -50,7 +51,7 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTPtr & node, const
else if (const auto * identifier = node->as<ASTIdentifier>())
*this = DatabaseAndTableWithAlias(*identifier, current_database);
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: identifier or table identifier expected");
throw Exception(ErrorCodes::INVALID_IDENTIFIER, "Identifier or table identifier expected");
}
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database)

View File

@ -31,6 +31,7 @@
#include <Storages/StorageInMemoryMetadata.h>
#include <Storages/WindowView/StorageWindowView.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/BlockNumberColumn.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
@ -95,7 +96,6 @@ namespace ErrorCodes
extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY;
extern const int ILLEGAL_SYNTAX_FOR_DATA_TYPE;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_INDEX;
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_DATABASE;
extern const int PATH_ACCESS_DENIED;
@ -698,8 +698,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
for (const auto & index : create.columns_list->indices->children)
{
IndexDescription index_desc = IndexDescription::getIndexFromAST(index->clone(), properties.columns, getContext());
if (properties.indices.has(index_desc.name))
throw Exception(ErrorCodes::ILLEGAL_INDEX, "Duplicated index name {}", backQuoteIfNeed(index_desc.name));
const auto & settings = getContext()->getSettingsRef();
if (index_desc.type == INVERTED_INDEX_NAME && !settings.allow_experimental_inverted_index)
{
@ -714,7 +712,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
properties.indices.push_back(index_desc);
}
if (create.columns_list->projections)
for (const auto & projection_ast : create.columns_list->projections->children)
{
@ -837,6 +834,13 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat
"Cannot create table with column '{}' for *MergeTree engines because it "
"is reserved for lightweight delete feature",
LightweightDeleteDescription::FILTER_COLUMN.name);
auto search_block_number = all_columns.find(BlockNumberColumn::name);
if (search_block_number != all_columns.end())
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Cannot create table with column '{}' for *MergeTree engines because it "
"is reserved for storing block number",
BlockNumberColumn::name);
}
const auto & settings = getContext()->getSettingsRef();

View File

@ -34,6 +34,7 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
extern const int INCORRECT_QUERY;
extern const int TABLE_IS_READ_ONLY;
extern const int TABLE_NOT_EMPTY;
}
namespace ActionLocks
@ -55,7 +56,8 @@ InterpreterDropQuery::InterpreterDropQuery(const ASTPtr & query_ptr_, ContextMut
BlockIO InterpreterDropQuery::execute()
{
auto & drop = query_ptr->as<ASTDropQuery &>();
if (!drop.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext()))
if (!drop.cluster.empty() && drop.table && !drop.if_empty && !maybeRemoveOnCluster(query_ptr, getContext()))
{
DDLQueryOnClusterParams params;
params.access_to_check = getRequiredAccessForDDLOnCluster();
@ -67,6 +69,12 @@ BlockIO InterpreterDropQuery::execute()
if (drop.table)
return executeToTable(drop);
else if (drop.database && !drop.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext()))
{
DDLQueryOnClusterParams params;
params.access_to_check = getRequiredAccessForDDLOnCluster();
return executeDDLQueryOnCluster(query_ptr, getContext(), params);
}
else if (drop.database)
return executeToDatabase(drop);
else
@ -122,6 +130,12 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue
if (database && table)
{
const auto & settings = getContext()->getSettingsRef();
if (query.if_empty)
{
if (auto rows = table->totalRows(settings); rows > 0)
throw Exception(ErrorCodes::TABLE_NOT_EMPTY, "Table {} is not empty", backQuoteIfNeed(table_id.table_name));
}
checkStorageSupportsTransactionsIfNeeded(table, context_);
auto & ast_drop_query = query.as<ASTDropQuery &>();
@ -151,6 +165,18 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue
else
drop_storage = AccessType::DROP_TABLE;
auto new_query_ptr = query.clone();
auto & query_to_send = new_query_ptr->as<ASTDropQuery &>();
if (!query.cluster.empty() && !maybeRemoveOnCluster(new_query_ptr, getContext()))
{
query_to_send.if_empty = false;
DDLQueryOnClusterParams params;
params.access_to_check = getRequiredAccessForDDLOnCluster();
return executeDDLQueryOnCluster(new_query_ptr, getContext(), params);
}
if (database->shouldReplicateQuery(getContext(), query_ptr))
{
if (query.kind == ASTDropQuery::Kind::Detach)
@ -162,7 +188,10 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue
ddl_guard->releaseTableLock();
table.reset();
return database->tryEnqueueReplicatedDDL(query.clone(), context_);
query_to_send.if_empty = false;
return database->tryEnqueueReplicatedDDL(new_query_ptr, context_);
}
if (query.kind == ASTDropQuery::Kind::Detach)
@ -340,9 +369,13 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query,
if (query.kind == ASTDropQuery::Kind::Detach && query.permanently)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DETACH PERMANENTLY is not implemented for databases");
if (query.if_empty)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DROP IF EMPTY is not implemented for databases");
if (database->hasReplicationThread())
database->stopReplication();
if (database->shouldBeEmptyOnDetach())
{
/// Cancel restarting replicas in that database, wait for remaining RESTART queries to finish.
@ -355,6 +388,7 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query,
if (truncate)
query_for_table.kind = ASTDropQuery::Kind::Drop;
query_for_table.if_exists = true;
query_for_table.if_empty = false;
query_for_table.setDatabase(database_name);
query_for_table.sync = query.sync;

View File

@ -7,6 +7,7 @@
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/StorageFromMergeTreeDataPart.h>
#include <Storages/StorageMergeTree.h>
#include <Storages/BlockNumberColumn.h>
#include <Processors/Transforms/FilterTransform.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/CreatingSetsTransform.h>
@ -40,7 +41,6 @@
#include <Parsers/makeASTForLogicalFunction.h>
#include <Common/logger_useful.h>
namespace DB
{
@ -56,6 +56,7 @@ namespace ErrorCodes
extern const int THERE_IS_NO_COLUMN;
}
namespace
{
@ -416,6 +417,12 @@ static void validateUpdateColumns(
found = true;
}
/// Dont allow to override value of block number virtual column
if (!found && column_name == BlockNumberColumn::name)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Update is not supported for virtual column {} ", backQuote(column_name));
}
if (!found)
{
for (const auto & col : metadata_snapshot->getColumns().getMaterialized())
@ -511,7 +518,8 @@ void MutationsInterpreter::prepare(bool dry_run)
for (const auto & [name, _] : command.column_to_update_expression)
{
if (!available_columns_set.contains(name) && name != LightweightDeleteDescription::FILTER_COLUMN.name)
if (!available_columns_set.contains(name) && name != LightweightDeleteDescription::FILTER_COLUMN.name
&& name != BlockNumberColumn::name)
throw Exception(ErrorCodes::THERE_IS_NO_COLUMN,
"Column {} is updated but not requested to read", name);
@ -613,6 +621,8 @@ void MutationsInterpreter::prepare(bool dry_run)
type = physical_column->type;
else if (column == LightweightDeleteDescription::FILTER_COLUMN.name)
type = LightweightDeleteDescription::FILTER_COLUMN.type;
else if (column == BlockNumberColumn::name)
type = BlockNumberColumn::type;
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown column {}", column);
@ -1087,6 +1097,18 @@ struct VirtualColumns
virtuals.emplace_back(ColumnAndPosition{.column = std::move(column), .position = i});
}
else if (columns_to_read[i] == BlockNumberColumn::name)
{
if (!part->getColumns().contains(BlockNumberColumn::name))
{
ColumnWithTypeAndName block_number_column;
block_number_column.type = BlockNumberColumn::type;
block_number_column.column = block_number_column.type->createColumnConst(0, part->info.min_block);
block_number_column.name = std::move(columns_to_read[i]);
virtuals.emplace_back(ColumnAndPosition{.column = std::move(block_number_column), .position = i});
}
}
}
if (!virtuals.empty())

View File

@ -48,7 +48,7 @@ static bool equals(const DataTypes & lhs, const DataTypes & rhs)
FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {}
SetPtr FutureSetFromStorage::get() const { return set; }
const DataTypes & FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); }
DataTypes FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); }
SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &)
{
@ -73,7 +73,7 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings)
set->finishInsert();
}
const DataTypes & FutureSetFromTuple::getTypes() const { return set->getElementsTypes(); }
DataTypes FutureSetFromTuple::getTypes() const { return set->getElementsTypes(); }
SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context)
{
@ -138,7 +138,7 @@ void FutureSetFromSubquery::setQueryPlan(std::unique_ptr<QueryPlan> source_)
set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName());
}
const DataTypes & FutureSetFromSubquery::getTypes() const
DataTypes FutureSetFromSubquery::getTypes() const
{
return set_and_key->set->getElementsTypes();
}
@ -183,7 +183,10 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context)
{
auto set = external_table_set->buildOrderedSetInplace(context);
if (set)
return set_and_key->set = set;
{
set_and_key->set = set;
return set_and_key->set;
}
}
auto plan = build(context);

View File

@ -47,7 +47,7 @@ public:
/// Returns set if set is ready (created and filled) or nullptr if not.
virtual SetPtr get() const = 0;
/// Returns set->getElementsTypes(), even if set is not created yet.
virtual const DataTypes & getTypes() const = 0;
virtual DataTypes getTypes() const = 0;
/// If possible, return set with stored elements useful for PK analysis.
virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0;
};
@ -62,7 +62,7 @@ public:
FutureSetFromStorage(SetPtr set_);
SetPtr get() const override;
const DataTypes & getTypes() const override;
DataTypes getTypes() const override;
SetPtr buildOrderedSetInplace(const ContextPtr &) override;
private:
@ -79,7 +79,7 @@ public:
SetPtr get() const override { return set; }
SetPtr buildOrderedSetInplace(const ContextPtr & context) override;
const DataTypes & getTypes() const override;
DataTypes getTypes() const override;
private:
SetPtr set;
@ -105,7 +105,7 @@ public:
const Settings & settings);
SetPtr get() const override;
const DataTypes & getTypes() const override;
DataTypes getTypes() const override;
SetPtr buildOrderedSetInplace(const ContextPtr & context) override;
std::unique_ptr<QueryPlan> build(const ContextPtr & context);

View File

@ -94,11 +94,12 @@ namespace DB
namespace ErrorCodes
{
extern const int CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS;
extern const int INTO_OUTFILE_NOT_ALLOWED;
extern const int QUERY_WAS_CANCELLED;
extern const int INVALID_TRANSACTION;
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
extern const int QUERY_WAS_CANCELLED;
}
@ -991,7 +992,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
if (!async_insert)
{
/// If it is a non-internal SELECT, and passive/read use of the query cache is enabled, and the cache knows the query, then set
/// If it is a non-internal SELECT, and passive (read) use of the query cache is enabled, and the cache knows the query, then set
/// a pipeline with a source populated by the query cache.
auto get_result_from_query_cache = [&]()
{
@ -1091,11 +1092,14 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
res = interpreter->execute();
/// If it is a non-internal SELECT query, and active/write use of the query cache is enabled, then add a processor on
/// If it is a non-internal SELECT query, and active (write) use of the query cache is enabled, then add a processor on
/// top of the pipeline which stores the result in the query cache.
if (can_use_query_cache && settings.enable_writes_to_query_cache
&& (!astContainsNonDeterministicFunctions(ast, context) || settings.query_cache_store_results_of_queries_with_nondeterministic_functions))
if (can_use_query_cache && settings.enable_writes_to_query_cache)
{
if (astContainsNonDeterministicFunctions(ast, context) && !settings.query_cache_store_results_of_queries_with_nondeterministic_functions)
throw Exception(ErrorCodes::CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS,
"Unable to cache the query result because the query contains a non-deterministic function. Use setting query_cache_store_results_of_queries_with_nondeterministic_functions = 1 to store the query result regardless.");
QueryCache::Key key(
ast, res.pipeline.getHeader(),
context->getUserName(), settings.query_cache_share_between_users,

View File

@ -20,6 +20,7 @@
#include <Columns/ColumnArray.h>
#include <DataTypes/DataTypeArray.h>
#include <Storages/StorageInMemoryMetadata.h>
#include <Storages/BlockNumberColumn.h>
namespace DB
@ -260,7 +261,7 @@ void fillMissingColumns(
const NamesAndTypesList & requested_columns,
const NamesAndTypesList & available_columns,
const NameSet & partially_read_columns,
StorageMetadataPtr metadata_snapshot)
StorageMetadataPtr metadata_snapshot, size_t block_number)
{
size_t num_columns = requested_columns.size();
if (num_columns != res_columns.size())
@ -339,9 +340,14 @@ void fillMissingColumns(
}
else
{
/// We must turn a constant column into a full column because the interpreter could infer
/// that it is constant everywhere but in some blocks (from other parts) it can be a full column.
res_columns[i] = type->createColumnConstWithDefaultValue(num_rows)->convertToFullColumnIfConst();
if (requested_column->name == BlockNumberColumn::name)
res_columns[i] = type->createColumnConst(num_rows, block_number)->convertToFullColumnIfConst();
else
/// We must turn a constant column into a full column because the interpreter could infer
/// that it is constant everywhere but in some blocks (from other parts) it can be a full column.
res_columns[i] = type->createColumnConstWithDefaultValue(num_rows)->convertToFullColumnIfConst();
}
}
}

View File

@ -46,6 +46,6 @@ void fillMissingColumns(
const NamesAndTypesList & requested_columns,
const NamesAndTypesList & available_columns,
const NameSet & partially_read_columns,
StorageMetadataPtr metadata_snapshot);
StorageMetadataPtr metadata_snapshot, size_t block_number = 0);
}

View File

@ -60,6 +60,9 @@ void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState
if (if_exists)
settings.ostr << "IF EXISTS ";
if (if_empty)
settings.ostr << "IF EMPTY ";
settings.ostr << (settings.hilite ? hilite_none : "");
if (!table && database)

View File

@ -21,6 +21,7 @@ public:
Kind kind;
bool if_exists{false};
bool if_empty{false};
/// Useful if we already have a DDL lock
bool no_ddl_lock{false};

View File

@ -19,6 +19,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons
ParserKeyword s_database("DATABASE");
ParserToken s_dot(TokenType::Dot);
ParserKeyword s_if_exists("IF EXISTS");
ParserKeyword s_if_empty("IF EMPTY");
ParserIdentifier name_p(true);
ParserKeyword s_permanently("PERMANENTLY");
ParserKeyword s_no_delay("NO DELAY");
@ -28,6 +29,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons
ASTPtr table;
String cluster_str;
bool if_exists = false;
bool if_empty = false;
bool temporary = false;
bool is_dictionary = false;
bool is_view = false;
@ -39,6 +41,9 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons
if (s_if_exists.ignore(pos, expected))
if_exists = true;
if (s_if_empty.ignore(pos, expected))
if_empty = true;
if (!name_p.parse(pos, database, expected))
return false;
}
@ -60,6 +65,9 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons
if (s_if_exists.ignore(pos, expected))
if_exists = true;
if (s_if_empty.ignore(pos, expected))
if_empty = true;
if (!name_p.parse(pos, table, expected))
return false;
@ -90,6 +98,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons
query->kind = kind;
query->if_exists = if_exists;
query->if_empty = if_empty;
query->temporary = temporary;
query->is_dictionary = is_dictionary;
query->is_view = is_view;

View File

@ -8,7 +8,10 @@ namespace DB
{
/** Query like this:
* DROP|DETACH|TRUNCATE TABLE [IF EXISTS] [db.]name [PERMANENTLY]
* DROP TABLE [IF EXISTS|EMPTY] [db.]name [PERMANENTLY]
*
* Or:
* DETACH|TRUNCATE TABLE [IF EXISTS] [db.]name [PERMANENTLY]
*
* Or:
* DROP DATABASE [IF EXISTS] db

View File

@ -7,77 +7,88 @@ namespace DB
{
bool parseIntervalKind(IParser::Pos & pos, Expected & expected, IntervalKind & result)
{
if (ParserKeyword("NANOSECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_NANOSECOND").ignore(pos, expected)
if (ParserKeyword("NANOSECOND").ignore(pos, expected) || ParserKeyword("NANOSECONDS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_NANOSECOND").ignore(pos, expected)
|| ParserKeyword("NS").ignore(pos, expected))
{
result = IntervalKind::Nanosecond;
return true;
}
if (ParserKeyword("MICROSECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_MICROSECOND").ignore(pos, expected)
if (ParserKeyword("MICROSECOND").ignore(pos, expected) || ParserKeyword("MICROSECONDS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_MICROSECOND").ignore(pos, expected)
|| ParserKeyword("MCS").ignore(pos, expected))
{
result = IntervalKind::Microsecond;
return true;
}
if (ParserKeyword("MILLISECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_MILLISECOND").ignore(pos, expected)
if (ParserKeyword("MILLISECOND").ignore(pos, expected) || ParserKeyword("MILLISECONDS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_MILLISECOND").ignore(pos, expected)
|| ParserKeyword("MS").ignore(pos, expected))
{
result = IntervalKind::Millisecond;
return true;
}
if (ParserKeyword("SECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_SECOND").ignore(pos, expected)
if (ParserKeyword("SECOND").ignore(pos, expected) || ParserKeyword("SECONDS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_SECOND").ignore(pos, expected)
|| ParserKeyword("SS").ignore(pos, expected) || ParserKeyword("S").ignore(pos, expected))
{
result = IntervalKind::Second;
return true;
}
if (ParserKeyword("MINUTE").ignore(pos, expected) || ParserKeyword("SQL_TSI_MINUTE").ignore(pos, expected)
if (ParserKeyword("MINUTE").ignore(pos, expected) || ParserKeyword("MINUTES").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_MINUTE").ignore(pos, expected)
|| ParserKeyword("MI").ignore(pos, expected) || ParserKeyword("N").ignore(pos, expected))
{
result = IntervalKind::Minute;
return true;
}
if (ParserKeyword("HOUR").ignore(pos, expected) || ParserKeyword("SQL_TSI_HOUR").ignore(pos, expected)
if (ParserKeyword("HOUR").ignore(pos, expected) || ParserKeyword("HOURS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_HOUR").ignore(pos, expected)
|| ParserKeyword("HH").ignore(pos, expected) || ParserKeyword("H").ignore(pos, expected))
{
result = IntervalKind::Hour;
return true;
}
if (ParserKeyword("DAY").ignore(pos, expected) || ParserKeyword("SQL_TSI_DAY").ignore(pos, expected)
if (ParserKeyword("DAY").ignore(pos, expected) || ParserKeyword("DAYS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_DAY").ignore(pos, expected)
|| ParserKeyword("DD").ignore(pos, expected) || ParserKeyword("D").ignore(pos, expected))
{
result = IntervalKind::Day;
return true;
}
if (ParserKeyword("WEEK").ignore(pos, expected) || ParserKeyword("SQL_TSI_WEEK").ignore(pos, expected)
if (ParserKeyword("WEEK").ignore(pos, expected) || ParserKeyword("WEEKS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_WEEK").ignore(pos, expected)
|| ParserKeyword("WK").ignore(pos, expected) || ParserKeyword("WW").ignore(pos, expected))
{
result = IntervalKind::Week;
return true;
}
if (ParserKeyword("MONTH").ignore(pos, expected) || ParserKeyword("SQL_TSI_MONTH").ignore(pos, expected)
if (ParserKeyword("MONTH").ignore(pos, expected) || ParserKeyword("MONTHS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_MONTH").ignore(pos, expected)
|| ParserKeyword("MM").ignore(pos, expected) || ParserKeyword("M").ignore(pos, expected))
{
result = IntervalKind::Month;
return true;
}
if (ParserKeyword("QUARTER").ignore(pos, expected) || ParserKeyword("SQL_TSI_QUARTER").ignore(pos, expected)
if (ParserKeyword("QUARTER").ignore(pos, expected) || ParserKeyword("QUARTERS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_QUARTER").ignore(pos, expected)
|| ParserKeyword("QQ").ignore(pos, expected) || ParserKeyword("Q").ignore(pos, expected))
{
result = IntervalKind::Quarter;
return true;
}
if (ParserKeyword("YEAR").ignore(pos, expected) || ParserKeyword("SQL_TSI_YEAR").ignore(pos, expected)
if (ParserKeyword("YEAR").ignore(pos, expected) || ParserKeyword("YEARS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_YEAR").ignore(pos, expected)
|| ParserKeyword("YYYY").ignore(pos, expected) || ParserKeyword("YY").ignore(pos, expected))
{
result = IntervalKind::Year;

View File

@ -130,7 +130,7 @@ static std::shared_ptr<parquet::FileMetaData> getFileMetadata(
const FormatSettings & format_settings,
std::atomic<int> & is_stopped)
{
auto arrow_file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES);
auto arrow_file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true);
return parquet::ReadMetaData(arrow_file);
}
@ -495,12 +495,15 @@ NamesAndTypesList ParquetMetadataSchemaReader::readSchema()
void registerInputFormatParquetMetadata(FormatFactory & factory)
{
factory.registerInputFormat(
factory.registerRandomAccessInputFormat(
"ParquetMetadata",
[](ReadBuffer &buf,
const Block &sample,
const RowInputFormatParams &,
const FormatSettings & settings)
[](ReadBuffer & buf,
const Block & sample,
const FormatSettings & settings,
const ReadSettings &,
bool /* is_remote_fs */,
size_t /* max_download_threads */,
size_t /* max_parsing_threads */)
{
return std::make_shared<ParquetMetadataInputFormat>(buf, sample, settings);
});

View File

@ -28,7 +28,6 @@ ReplacingSortedAlgorithm::ReplacingSortedAlgorithm(
, cleanup(cleanup_)
, cleanedup_rows_count(cleanedup_rows_count_)
{
if (!is_deleted_column.empty())
is_deleted_column_number = header_.getPositionByName(is_deleted_column);
if (!version_column.empty())
@ -83,8 +82,11 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge()
uint8_t value = assert_cast<const ColumnUInt8 &>(*(*selected_row.all_columns)[is_deleted_column_number]).getData()[selected_row.row_num];
if (!cleanup || !value)
insertRow();
else if (cleanedup_rows_count != nullptr)
else if (cleanup && cleanedup_rows_count != nullptr)
{
*cleanedup_rows_count += current_row_sources.size();
current_row_sources.resize(0);
}
}
else
insertRow();
@ -141,8 +143,11 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge()
uint8_t value = assert_cast<const ColumnUInt8 &>(*(*selected_row.all_columns)[is_deleted_column_number]).getData()[selected_row.row_num];
if (!cleanup || !value)
insertRow();
else if (cleanedup_rows_count != nullptr)
else if (cleanup && cleanedup_rows_count != nullptr)
{
*cleanedup_rows_count += current_row_sources.size();
current_row_sources.resize(0);
}
}
else
insertRow();

View File

@ -12,6 +12,7 @@
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <IO/WriteHelpers.h>
#include <Storages/BlockNumberColumn.h>
namespace DB
@ -222,6 +223,12 @@ static SummingSortedAlgorithm::ColumnsDefinition defineColumns(
const ColumnWithTypeAndName & column = header.safeGetByPosition(i);
const auto * simple = dynamic_cast<const DataTypeCustomSimpleAggregateFunction *>(column.type->getCustomName());
if (column.name == BlockNumberColumn::name)
{
def.column_numbers_not_to_aggregate.push_back(i);
continue;
}
/// Discover nested Maps and find columns for summation
if (typeid_cast<const DataTypeArray *>(column.type.get()) && !simple)
{

View File

@ -236,28 +236,29 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact
scalars["_shard_num"]
= Block{{DataTypeUInt32().createColumnConst(1, shard.shard_info.shard_num), std::make_shared<DataTypeUInt32>(), "_shard_num"}};
if (context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS)
if (context->canUseParallelReplicas())
{
if (context->getSettingsRef().cluster_for_parallel_replicas.changed)
{
const String cluster_for_parallel_replicas = context->getSettingsRef().cluster_for_parallel_replicas;
if (cluster_for_parallel_replicas != cluster_name)
LOG_INFO(log, "cluster_for_parallel_replicas has been set for the query but has no effect: {}. Distributed table cluster is used: {}",
cluster_for_parallel_replicas, cluster_name);
LOG_INFO(
log,
"cluster_for_parallel_replicas has been set for the query but has no effect: {}. Distributed table cluster is "
"used: {}",
cluster_for_parallel_replicas,
cluster_name);
}
LOG_TRACE(&Poco::Logger::get("ReadFromRemote"), "Setting `cluster_for_parallel_replicas` to {}", cluster_name);
LOG_TRACE(log, "Setting `cluster_for_parallel_replicas` to {}", cluster_name);
context->setSetting("cluster_for_parallel_replicas", cluster_name);
}
std::shared_ptr<RemoteQueryExecutor> remote_query_executor;
remote_query_executor = std::make_shared<RemoteQueryExecutor>(
shard.shard_info.pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage);
auto remote_query_executor = std::make_shared<RemoteQueryExecutor>(
shard.shard_info.pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage);
remote_query_executor->setLogger(log);
if (context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS)
if (context->canUseParallelReplicas())
{
// when doing parallel reading from replicas (ParallelReplicasMode::READ_TASKS) on a shard:
// establish a connection to a replica on the shard, the replica will instantiate coordinator to manage parallel reading from replicas on the shard.
@ -273,7 +274,8 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact
if (!table_func_ptr)
remote_query_executor->setMainTable(shard.main_table ? shard.main_table : main_table);
pipes.emplace_back(createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending));
pipes.emplace_back(
createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending));
addConvertingActions(pipes.back(), output_stream->header);
}

View File

@ -49,7 +49,8 @@ TTLTransform::TTLTransform(
for (const auto & group_by_ttl : metadata_snapshot_->getGroupByTTLs())
algorithms.emplace_back(std::make_unique<TTLAggregationAlgorithm>(
group_by_ttl, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_, getInputPort().getHeader(), storage_));
group_by_ttl, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_,
getInputPort().getHeader(), storage_));
if (metadata_snapshot_->hasAnyColumnTTL())
{

View File

@ -382,9 +382,9 @@ void KeeperTCPHandler::runImpl()
}
auto response_fd = poll_wrapper->getResponseFD();
auto response_callback = [this, response_fd] (const Coordination::ZooKeeperResponsePtr & response)
auto response_callback = [responses_ = this->responses, response_fd](const Coordination::ZooKeeperResponsePtr & response)
{
if (!responses->push(response))
if (!responses_->push(response))
throw Exception(ErrorCodes::SYSTEM_ERROR,
"Could not push response with xid {} and zxid {}",
response->xid,

View File

@ -25,7 +25,7 @@ struct SocketInterruptablePollWrapper;
using SocketInterruptablePollWrapperPtr = std::unique_ptr<SocketInterruptablePollWrapper>;
using ThreadSafeResponseQueue = ConcurrentBoundedQueue<Coordination::ZooKeeperResponsePtr>;
using ThreadSafeResponseQueuePtr = std::unique_ptr<ThreadSafeResponseQueue>;
using ThreadSafeResponseQueuePtr = std::shared_ptr<ThreadSafeResponseQueue>;
struct LastOp;
using LastOpMultiVersion = MultiVersion<LastOp>;

View File

@ -27,6 +27,7 @@
#include <Storages/AlterCommands.h>
#include <Storages/IStorage.h>
#include <Storages/LightweightDeleteDescription.h>
#include <Storages/BlockNumberColumn.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Common/typeid_cast.h>
#include <Common/randomSeed.h>
@ -782,7 +783,7 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada
/// Drop alias is metadata alter, in other case mutation is required.
if (type == DROP_COLUMN)
return metadata.columns.hasColumnOrNested(GetColumnsOptions::AllPhysical, column_name) ||
column_name == LightweightDeleteDescription::FILTER_COLUMN.name;
column_name == LightweightDeleteDescription::FILTER_COLUMN.name || column_name == BlockNumberColumn::name;
if (type != MODIFY_COLUMN || data_type == nullptr)
return false;
@ -1066,6 +1067,10 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add column {}: "
"this column name is reserved for lightweight delete feature", backQuote(column_name));
if (column_name == BlockNumberColumn::name && std::dynamic_pointer_cast<MergeTreeData>(table))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add column {}: "
"this column name is reserved for _block_number persisting feature", backQuote(column_name));
if (command.codec)
CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs, context->getSettingsRef().enable_deflate_qpl_codec);
@ -1270,6 +1275,10 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot rename to {}: "
"this column name is reserved for lightweight delete feature", backQuote(command.rename_to));
if (command.rename_to == BlockNumberColumn::name && std::dynamic_pointer_cast<MergeTreeData>(table))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot rename to {}: "
"this column name is reserved for _block_number persisting feature", backQuote(command.rename_to));
if (modified_columns.contains(column_name))
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot rename and modify the same column {} "
"in a single ALTER query", backQuote(column_name));

View File

@ -0,0 +1,23 @@
#include <Storages/BlockNumberColumn.h>
#include <Compression/CompressionCodecMultiple.h>
namespace DB
{
CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size);
CompressionCodecPtr getCompressionCodecForBlockNumberColumn()
{
std::vector <CompressionCodecPtr> codecs;
codecs.reserve(2);
auto data_bytes_size = BlockNumberColumn::type->getSizeOfValueInMemory();
codecs.emplace_back(getCompressionCodecDelta(data_bytes_size));
codecs.emplace_back(CompressionCodecFactory::instance().get("LZ4", {}));
return std::make_shared<CompressionCodecMultiple>(codecs);
}
const String BlockNumberColumn::name = "_block_number";
const DataTypePtr BlockNumberColumn::type = std::make_shared<DataTypeUInt64>();
const CompressionCodecPtr BlockNumberColumn::compression_codec = getCompressionCodecForBlockNumberColumn();
}

View File

@ -0,0 +1,16 @@
#pragma once
#include <Core/NamesAndTypes.h>
#include <DataTypes/DataTypesNumber.h>
#include <Compression/CompressionFactory.h>
namespace DB
{
struct BlockNumberColumn
{
static const String name;
static const DataTypePtr type;
static const CompressionCodecPtr compression_codec;
};
}

View File

@ -30,11 +30,15 @@
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/FunctionNameNormalizer.h>
#include <Storages/BlockNumberColumn.h>
namespace DB
{
CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size);
namespace ErrorCodes
{
extern const int NO_SUCH_COLUMN_IN_TABLE;
@ -721,11 +725,13 @@ CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_
CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_name) const
{
assert (column_name != BlockNumberColumn::name);
return getCodecOrDefault(column_name, CompressionCodecFactory::instance().getDefaultCodec());
}
ASTPtr ColumnsDescription::getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) const
{
assert (column_name != BlockNumberColumn::name);
const auto it = columns.get<1>().find(column_name);
if (it == columns.get<1>().end() || !it->codec)

View File

@ -416,7 +416,8 @@ void DataPartStorageOnDiskBase::backup(
MutableDataPartStoragePtr DataPartStorageOnDiskBase::freeze(
const std::string & to,
const std::string & dir_path,
const WriteSettings & settings,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
std::function<void(const DiskPtr &)> save_metadata_callback,
const ClonePartParams & params) const
{
@ -430,7 +431,8 @@ MutableDataPartStoragePtr DataPartStorageOnDiskBase::freeze(
disk,
getRelativePath(),
fs::path(to) / dir_path,
settings,
read_settings,
write_settings,
params.make_source_readonly,
/* max_level= */ {},
params.copy_instead_of_hardlink,
@ -466,6 +468,7 @@ MutableDataPartStoragePtr DataPartStorageOnDiskBase::clonePart(
const std::string & to,
const std::string & dir_path,
const DiskPtr & dst_disk,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
Poco::Logger * log) const
{
@ -482,7 +485,7 @@ MutableDataPartStoragePtr DataPartStorageOnDiskBase::clonePart(
try
{
dst_disk->createDirectories(to);
src_disk->copyDirectoryContent(getRelativePath(), dst_disk, path_to_clone, write_settings);
src_disk->copyDirectoryContent(getRelativePath(), dst_disk, path_to_clone, read_settings, write_settings);
}
catch (...)
{

View File

@ -63,7 +63,8 @@ public:
MutableDataPartStoragePtr freeze(
const std::string & to,
const std::string & dir_path,
const WriteSettings & settings,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
std::function<void(const DiskPtr &)> save_metadata_callback,
const ClonePartParams & params) const override;
@ -71,6 +72,7 @@ public:
const std::string & to,
const std::string & dir_path,
const DiskPtr & dst_disk,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
Poco::Logger * log) const override;

View File

@ -250,7 +250,8 @@ public:
virtual std::shared_ptr<IDataPartStorage> freeze(
const std::string & to,
const std::string & dir_path,
const WriteSettings & settings,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
std::function<void(const DiskPtr &)> save_metadata_callback,
const ClonePartParams & params) const = 0;
@ -259,6 +260,7 @@ public:
const std::string & to,
const std::string & dir_path,
const DiskPtr & disk,
const ReadSettings & read_settings,
const WriteSettings & write_settings,
Poco::Logger * log) const = 0;

View File

@ -1793,12 +1793,13 @@ DataPartStoragePtr IMergeTreeDataPart::makeCloneInDetached(const String & prefix
return getDataPartStorage().freeze(
storage.relative_data_path,
*maybe_path_in_detached,
Context::getGlobalContextInstance()->getReadSettings(),
Context::getGlobalContextInstance()->getWriteSettings(),
/* save_metadata_callback= */ {},
params);
}
MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & disk, const String & directory_name, const WriteSettings & write_settings) const
MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & disk, const String & directory_name, const ReadSettings & read_settings, const WriteSettings & write_settings) const
{
assertOnDisk();
@ -1808,7 +1809,7 @@ MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & di
throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not clone data part {} to empty directory.", name);
String path_to_clone = fs::path(storage.relative_data_path) / directory_name / "";
return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, write_settings, storage.log);
return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, read_settings, write_settings, storage.log);
}
UInt64 IMergeTreeDataPart::getIndexSizeFromFile() const

View File

@ -377,7 +377,7 @@ public:
const DiskTransactionPtr & disk_transaction) const;
/// Makes full clone of part in specified subdirectory (relative to storage data directory, e.g. "detached") on another disk
MutableDataPartStoragePtr makeCloneOnDisk(const DiskPtr & disk, const String & directory_name, const WriteSettings & write_settings) const;
MutableDataPartStoragePtr makeCloneOnDisk(const DiskPtr & disk, const String & directory_name, const ReadSettings & read_settings, const WriteSettings & write_settings) const;
/// Checks that .bin and .mrk files exist.
///
@ -477,10 +477,6 @@ public:
/// Moar hardening: this method is supposed to be used for debug assertions
bool assertHasValidVersionMetadata() const;
/// Return hardlink count for part.
/// Required for keep data on remote FS when part has shadow copies.
UInt32 getNumberOfRefereneces() const;
/// True if the part supports lightweight delete mutate.
bool supportLightweightDeleteMutate() const;

View File

@ -62,7 +62,7 @@ const IMergeTreeReader::ValueSizeMap & IMergeTreeReader::getAvgValueSizeHints()
return avg_value_size_hints;
}
void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows) const
void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows, size_t block_number) const
{
try
{
@ -71,7 +71,7 @@ void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_e
res_columns, num_rows,
Nested::convertToSubcolumns(requested_columns),
Nested::convertToSubcolumns(available_columns),
partially_read_columns, storage_snapshot->metadata);
partially_read_columns, storage_snapshot->metadata, block_number);
should_evaluate_missing_defaults = std::any_of(
res_columns.begin(), res_columns.end(), [](const auto & column) { return column == nullptr; });

View File

@ -45,7 +45,7 @@ public:
/// Add columns from ordered_names that are not present in the block.
/// Missing columns are added in the order specified by ordered_names.
/// num_rows is needed in case if all res_columns are nullptr.
void fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows) const;
void fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows, size_t block_number = 0) const;
/// Evaluate defaulted columns if necessary.
void evaluateMissingDefaults(Block additional_columns, Columns & res_columns) const;

View File

@ -3,6 +3,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/FieldToDataType.h>
#include <DataTypes/getLeastSupertype.h>
#include <DataTypes/Utils.h>
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
@ -1258,10 +1259,18 @@ bool KeyCondition::tryPrepareSetIndex(
const auto right_arg = func.getArgumentAt(1);
auto future_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types);
auto future_set = right_arg.tryGetPreparedSet();
if (!future_set)
return false;
const auto set_types = future_set->getTypes();
size_t set_types_size = set_types.size();
size_t indexes_mapping_size = indexes_mapping.size();
for (auto & index_mapping : indexes_mapping)
if (index_mapping.tuple_index >= set_types_size)
return false;
auto prepared_set = future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext());
if (!prepared_set)
return false;
@ -1270,11 +1279,72 @@ bool KeyCondition::tryPrepareSetIndex(
if (!prepared_set->hasExplicitSetElements())
return false;
prepared_set->checkColumnsNumber(left_args_count);
for (size_t i = 0; i < indexes_mapping.size(); ++i)
prepared_set->checkTypesEqual(indexes_mapping[i].tuple_index, data_types[i]);
/** Try to convert set columns to primary key columns.
* Example: SELECT id FROM test_table WHERE id IN (SELECT 1);
* In this example table `id` column has type UInt64, Set column has type UInt8. To use index
* we need to convert set column to primary key column.
*/
auto set_columns = prepared_set->getSetElements();
assert(set_types_size == set_columns.size());
out.set_index = std::make_shared<MergeTreeSetIndex>(prepared_set->getSetElements(), std::move(indexes_mapping));
for (size_t indexes_mapping_index = 0; indexes_mapping_index < indexes_mapping_size; ++indexes_mapping_index)
{
const auto & key_column_type = data_types[indexes_mapping_index];
size_t set_element_index = indexes_mapping[indexes_mapping_index].tuple_index;
auto set_element_type = set_types[set_element_index];
auto set_column = set_columns[set_element_index];
if (canBeSafelyCasted(set_element_type, key_column_type))
{
set_columns[set_element_index] = castColumn({set_column, set_element_type, {}}, key_column_type);
continue;
}
if (!key_column_type->canBeInsideNullable())
return false;
const NullMap * set_column_null_map = nullptr;
if (isNullableOrLowCardinalityNullable(set_element_type))
{
if (WhichDataType(set_element_type).isLowCardinality())
{
set_element_type = removeLowCardinality(set_element_type);
set_column = set_column->convertToFullColumnIfLowCardinality();
}
set_element_type = removeNullable(set_element_type);
const auto & set_column_nullable = assert_cast<const ColumnNullable &>(*set_column);
set_column_null_map = &set_column_nullable.getNullMapData();
set_column = set_column_nullable.getNestedColumnPtr();
}
auto nullable_set_column = castColumnAccurateOrNull({set_column, set_element_type, {}}, key_column_type);
const auto & nullable_set_column_typed = assert_cast<const ColumnNullable &>(*nullable_set_column);
const auto & nullable_set_column_null_map = nullable_set_column_typed.getNullMapData();
size_t nullable_set_column_null_map_size = nullable_set_column_null_map.size();
IColumn::Filter filter(nullable_set_column_null_map_size);
if (set_column_null_map)
{
for (size_t i = 0; i < nullable_set_column_null_map_size; ++i)
filter[i] = (*set_column_null_map)[i] || !nullable_set_column_null_map[i];
set_column = nullable_set_column_typed.filter(filter, 0);
}
else
{
for (size_t i = 0; i < nullable_set_column_null_map_size; ++i)
filter[i] = !nullable_set_column_null_map[i];
set_column = nullable_set_column_typed.getNestedColumn().filter(filter, 0);
}
set_columns[set_element_index] = std::move(set_column);
}
out.set_index = std::make_shared<MergeTreeSetIndex>(set_columns, std::move(indexes_mapping));
return true;
}

View File

@ -218,6 +218,14 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare()
ctx->need_remove_expired_values = false;
ctx->force_ttl = false;
if (supportsBlockNumberColumn(global_ctx) && !global_ctx->storage_columns.contains(BlockNumberColumn::name))
{
global_ctx->storage_columns.emplace_back(NameAndTypePair{BlockNumberColumn::name,BlockNumberColumn::type});
global_ctx->all_column_names.emplace_back(BlockNumberColumn::name);
global_ctx->gathering_columns.emplace_back(NameAndTypePair{BlockNumberColumn::name,BlockNumberColumn::type});
global_ctx->gathering_column_names.emplace_back(BlockNumberColumn::name);
}
SerializationInfo::Settings info_settings =
{
.ratio_of_defaults_for_sparse = global_ctx->data->getSettings()->ratio_of_defaults_for_sparse_serialization,
@ -251,12 +259,12 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare()
}
}
global_ctx->new_data_part->setColumns(global_ctx->storage_columns, infos, global_ctx->metadata_snapshot->getMetadataVersion());
const auto & local_part_min_ttl = global_ctx->new_data_part->ttl_infos.part_min_ttl;
if (local_part_min_ttl && local_part_min_ttl <= global_ctx->time_of_merge)
ctx->need_remove_expired_values = true;
global_ctx->new_data_part->setColumns(global_ctx->storage_columns, infos, global_ctx->metadata_snapshot->getMetadataVersion());
if (ctx->need_remove_expired_values && global_ctx->ttl_merges_blocker->isCancelled())
{
LOG_INFO(ctx->log, "Part {} has values with expired TTL, but merges with TTL are cancelled.", global_ctx->new_data_part->name);
@ -998,6 +1006,17 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
if (global_ctx->deduplicate)
{
/// We don't want to deduplicate by block number column
/// so if deduplicate_by_columns is empty, add all columns except _block_number
if (supportsBlockNumberColumn(global_ctx) && global_ctx->deduplicate_by_columns.empty())
{
for (const auto & col : global_ctx->merging_column_names)
{
if (col != BlockNumberColumn::name)
global_ctx->deduplicate_by_columns.emplace_back(col);
}
}
if (DistinctSortedTransform::isApplicable(header, sort_description, global_ctx->deduplicate_by_columns))
res_pipe.addTransform(std::make_shared<DistinctSortedTransform>(
res_pipe.getHeader(), sort_description, SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns));

View File

@ -13,6 +13,7 @@
#include <QueryPipeline/QueryPipeline.h>
#include <Compression/CompressedReadBufferFromFile.h>
#include <Common/filesystemHelpers.h>
#include <Storages/BlockNumberColumn.h>
#include <memory>
#include <list>
@ -388,6 +389,12 @@ private:
Stages::iterator stages_iterator = stages.begin();
/// Check for persisting block number column
static bool supportsBlockNumberColumn(GlobalRuntimeContextPtr global_ctx)
{
return global_ctx->data->getSettings()->allow_experimental_block_number_column && global_ctx->metadata_snapshot->getGroupByTTLs().empty();
}
};
/// FIXME

View File

@ -78,6 +78,7 @@
#include <Storages/VirtualColumnUtils.h>
#include <Storages/MergeTree/MergeTreeDataPartBuilder.h>
#include <Storages/MutationCommands.h>
#include <Storages/BlockNumberColumn.h>
#include <boost/range/algorithm_ext/erase.hpp>
#include <boost/algorithm/string/join.hpp>
@ -3730,7 +3731,7 @@ void MergeTreeData::checkPartDynamicColumns(MutableDataPartPtr & part, DataParts
const auto & part_columns = part->getColumns();
for (const auto & part_column : part_columns)
{
if (part_column.name == LightweightDeleteDescription::FILTER_COLUMN.name)
if (part_column.name == LightweightDeleteDescription::FILTER_COLUMN.name || part_column.name == BlockNumberColumn::name)
continue;
auto storage_column = columns.getPhysical(part_column.name);
@ -4967,7 +4968,7 @@ void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String &
throw Exception(ErrorCodes::UNKNOWN_DISK, "All parts of partition '{}' are already on disk '{}'", partition_id, disk->getName());
}
MovePartsOutcome moves_outcome = movePartsToSpace(parts, std::static_pointer_cast<Space>(disk), local_context->getWriteSettings());
MovePartsOutcome moves_outcome = movePartsToSpace(parts, std::static_pointer_cast<Space>(disk), local_context->getReadSettings(), local_context->getWriteSettings());
switch (moves_outcome)
{
case MovePartsOutcome::MovesAreCancelled:
@ -5030,7 +5031,7 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String
throw Exception(ErrorCodes::UNKNOWN_DISK, "All parts of partition '{}' are already on volume '{}'", partition_id, volume->getName());
}
MovePartsOutcome moves_outcome = movePartsToSpace(parts, std::static_pointer_cast<Space>(volume), local_context->getWriteSettings());
MovePartsOutcome moves_outcome = movePartsToSpace(parts, std::static_pointer_cast<Space>(volume), local_context->getReadSettings(), local_context->getWriteSettings());
switch (moves_outcome)
{
case MovePartsOutcome::MovesAreCancelled:
@ -7487,6 +7488,7 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::cloneAn
const MergeTreePartInfo & dst_part_info,
const StorageMetadataPtr & metadata_snapshot,
const IDataPartStorage::ClonePartParams & params,
const ReadSettings & read_settings,
const WriteSettings & write_settings)
{
/// Check that the storage policy contains the disk where the src_part is located.
@ -7544,6 +7546,7 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::cloneAn
auto dst_part_storage = src_part_storage->freeze(
relative_data_path,
tmp_dst_part_name,
read_settings,
write_settings,
/* save_metadata_callback= */ {},
params);
@ -7802,6 +7805,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher(
auto new_storage = data_part_storage->freeze(
backup_part_path,
part->getDataPartStorage().getPartDirectory(),
local_context->getReadSettings(),
local_context->getWriteSettings(),
callback,
params);
@ -8001,8 +8005,9 @@ bool MergeTreeData::scheduleDataMovingJob(BackgroundJobsAssignee & assignee)
assignee.scheduleMoveTask(std::make_shared<ExecutableLambdaAdapter>(
[this, moving_tagger] () mutable
{
ReadSettings read_settings = Context::getGlobalContextInstance()->getReadSettings();
WriteSettings write_settings = Context::getGlobalContextInstance()->getWriteSettings();
return moveParts(moving_tagger, write_settings, /* wait_for_move_if_zero_copy= */ false) == MovePartsOutcome::PartsMoved;
return moveParts(moving_tagger, read_settings, write_settings, /* wait_for_move_if_zero_copy= */ false) == MovePartsOutcome::PartsMoved;
}, moves_assignee_trigger, getStorageID()));
return true;
}
@ -8017,7 +8022,7 @@ bool MergeTreeData::areBackgroundMovesNeeded() const
return policy->getVolumes().size() == 1 && policy->getVolumes()[0]->getDisks().size() > 1;
}
MovePartsOutcome MergeTreeData::movePartsToSpace(const DataPartsVector & parts, SpacePtr space, const WriteSettings & write_settings)
MovePartsOutcome MergeTreeData::movePartsToSpace(const DataPartsVector & parts, SpacePtr space, const ReadSettings & read_settings, const WriteSettings & write_settings)
{
if (parts_mover.moves_blocker.isCancelled())
return MovePartsOutcome::MovesAreCancelled;
@ -8026,7 +8031,7 @@ MovePartsOutcome MergeTreeData::movePartsToSpace(const DataPartsVector & parts,
if (moving_tagger->parts_to_move.empty())
return MovePartsOutcome::NothingToMove;
return moveParts(moving_tagger, write_settings, /* wait_for_move_if_zero_copy= */ true);
return moveParts(moving_tagger, read_settings, write_settings, /* wait_for_move_if_zero_copy= */ true);
}
MergeTreeData::CurrentlyMovingPartsTaggerPtr MergeTreeData::selectPartsForMove()
@ -8081,7 +8086,7 @@ MergeTreeData::CurrentlyMovingPartsTaggerPtr MergeTreeData::checkPartsForMove(co
return std::make_shared<CurrentlyMovingPartsTagger>(std::move(parts_to_move), *this);
}
MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagger, const WriteSettings & write_settings, bool wait_for_move_if_zero_copy)
MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagger, const ReadSettings & read_settings, const WriteSettings & write_settings, bool wait_for_move_if_zero_copy)
{
LOG_INFO(log, "Got {} parts to move.", moving_tagger->parts_to_move.size());
@ -8142,7 +8147,7 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr &
{
if (lock->isLocked())
{
cloned_part = parts_mover.clonePart(moving_part, write_settings);
cloned_part = parts_mover.clonePart(moving_part, read_settings, write_settings);
parts_mover.swapClonedPart(cloned_part);
break;
}
@ -8169,7 +8174,7 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr &
}
else /// Ordinary move as it should be
{
cloned_part = parts_mover.clonePart(moving_part, write_settings);
cloned_part = parts_mover.clonePart(moving_part, read_settings, write_settings);
parts_mover.swapClonedPart(cloned_part);
}
write_part_log({});
@ -8269,6 +8274,7 @@ NamesAndTypesList MergeTreeData::getVirtuals() const
NameAndTypePair("_sample_factor", std::make_shared<DataTypeFloat64>()),
NameAndTypePair("_part_offset", std::make_shared<DataTypeUInt64>()),
LightweightDeleteDescription::FILTER_COLUMN,
NameAndTypePair(BlockNumberColumn::name, BlockNumberColumn::type),
};
}

View File

@ -848,6 +848,7 @@ public:
const MergeTreePartInfo & dst_part_info,
const StorageMetadataPtr & metadata_snapshot,
const IDataPartStorage::ClonePartParams & params,
const ReadSettings & read_settings,
const WriteSettings & write_settings);
virtual std::vector<MergeTreeMutationStatus> getMutationsStatus() const = 0;
@ -1340,7 +1341,7 @@ protected:
/// MergeTree because they store mutations in different way.
virtual std::map<int64_t, MutationCommands> getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0;
/// Moves part to specified space, used in ALTER ... MOVE ... queries
MovePartsOutcome movePartsToSpace(const DataPartsVector & parts, SpacePtr space, const WriteSettings & write_settings);
MovePartsOutcome movePartsToSpace(const DataPartsVector & parts, SpacePtr space, const ReadSettings & read_settings, const WriteSettings & write_settings);
struct PartBackupEntries
{
@ -1494,7 +1495,7 @@ private:
using CurrentlyMovingPartsTaggerPtr = std::shared_ptr<CurrentlyMovingPartsTagger>;
/// Move selected parts to corresponding disks
MovePartsOutcome moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagger, const WriteSettings & write_settings, bool wait_for_move_if_zero_copy);
MovePartsOutcome moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagger, const ReadSettings & read_settings, const WriteSettings & write_settings, bool wait_for_move_if_zero_copy);
/// Select parts for move and disks for them. Used in background moving processes.
CurrentlyMovingPartsTaggerPtr selectPartsForMove();

View File

@ -5,6 +5,7 @@
#include <Interpreters/Context.h>
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
#include <Compression/CompressedReadBufferFromFile.h>
#include <Storages/BlockNumberColumn.h>
namespace DB
@ -64,6 +65,12 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter(
ordered_columns_list.sort([this](const auto & lhs, const auto & rhs)
{ return *getColumnPosition(lhs.name) < *getColumnPosition(rhs.name); });
/// _block_number column is not added by user, but is persisted in a part after merge
/// If _block_number is not present in the parts to be merged, then it won't have a position
/// So check if its not present and add it at the end
if (columns_list.contains(BlockNumberColumn::name) && !ordered_columns_list.contains(BlockNumberColumn::name))
ordered_columns_list.emplace_back(NameAndTypePair{BlockNumberColumn::name, BlockNumberColumn::type});
return std::make_unique<MergeTreeDataPartWriterCompact>(
shared_from_this(), ordered_columns_list, metadata_snapshot,
indices_to_recalc, getMarksFileExtension(),

View File

@ -1,9 +1,12 @@
#include <Storages/MergeTree/MergeTreeDataPartWriterCompact.h>
#include <Storages/MergeTree/MergeTreeDataPartCompact.h>
#include <Storages/BlockNumberColumn.h>
namespace DB
{
CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size);
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
@ -53,7 +56,14 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact(
const auto & storage_columns = metadata_snapshot->getColumns();
for (const auto & column : columns_list)
addStreams(column, storage_columns.getCodecDescOrDefault(column.name, default_codec));
{
ASTPtr compression;
if (column.name == BlockNumberColumn::name)
compression = BlockNumberColumn::compression_codec->getFullCodecDesc();
else
compression = storage_columns.getCodecDescOrDefault(column.name, default_codec);
addStreams(column, compression);
}
}
void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & column, const ASTPtr & effective_codec_desc)

View File

@ -6,9 +6,12 @@
#include <Common/escapeForFileName.h>
#include <Columns/ColumnSparse.h>
#include <Common/logger_useful.h>
#include <Storages/BlockNumberColumn.h>
namespace DB
{
CompressionCodecPtr getCompressionCodecDelta(UInt8 delta_bytes_size);
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
@ -87,7 +90,14 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide(
{
const auto & columns = metadata_snapshot->getColumns();
for (const auto & it : columns_list)
addStreams(it, columns.getCodecDescOrDefault(it.name, default_codec));
{
ASTPtr compression;
if (it.name == BlockNumberColumn::name)
compression = BlockNumberColumn::compression_codec->getFullCodecDesc();
else
compression = columns.getCodecDescOrDefault(it.name, default_codec);
addStreams(it, compression);
}
}
void MergeTreeDataPartWriterWide::addStreams(

View File

@ -46,7 +46,7 @@
#include <Functions/IFunction.h>
#include <IO/WriteBufferFromOStream.h>
#include <Storages/BlockNumberColumn.h>
#include <Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h>
namespace CurrentMetrics
@ -1232,6 +1232,10 @@ static void selectColumnNames(
{
virt_column_names.push_back(name);
}
else if (name == BlockNumberColumn::name)
{
virt_column_names.push_back(name);
}
else if (name == "_part_uuid")
{
virt_column_names.push_back(name);

View File

@ -154,36 +154,45 @@ void MergeTreeIndexAggregatorAnnoy<Distance>::update(const Block & block, size_t
if (const auto & column_array = typeid_cast<const ColumnArray *>(column_cut.get()))
{
const auto & data = column_array->getData();
const auto & array = typeid_cast<const ColumnFloat32 &>(data).getData();
const auto & column_array_data = column_array->getData();
const auto & column_arary_data_float_data = typeid_cast<const ColumnFloat32 &>(column_array_data).getData();
if (array.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Array has 0 rows, {} rows expected", rows_read);
const auto & column_array_offsets = column_array->getOffsets();
const size_t num_rows = column_array_offsets.size();
const auto & offsets = column_array->getOffsets();
const size_t num_rows = offsets.size();
/// The Annoy algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays
/// are INSERTed into an Annoy-indexed column or if no value was specified at all in which case the arrays take on their default
/// value which is also empty.
if (column_array->isDefaultAt(0))
throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name);
/// Check all sizes are the same
size_t size = offsets[0];
size_t dimension = column_array_offsets[0];
for (size_t i = 0; i < num_rows - 1; ++i)
if (offsets[i + 1] - offsets[i] != size)
throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name);
if (column_array_offsets[i + 1] - column_array_offsets[i] != dimension)
throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name);
/// Also check that previously inserted blocks have the same size as this block.
/// Note that this guarantees consistency of dimension only within parts. We are unable to detect inconsistent dimensions across
/// parts - for this, a little help from the user is needed, e.g. CONSTRAINT cnstr CHECK length(array) = 42.
if (index && index->getDimensions() != dimension)
throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column '{}' must have equal length", index_column_name);
if (!index)
index = std::make_shared<AnnoyIndexWithSerialization<Distance>>(size);
index = std::make_shared<AnnoyIndexWithSerialization<Distance>>(dimension);
/// Add all rows of block
index->add_item(index->get_n_items(), array.data());
index->add_item(index->get_n_items(), column_arary_data_float_data.data());
for (size_t current_row = 1; current_row < num_rows; ++current_row)
index->add_item(index->get_n_items(), &array[offsets[current_row - 1]]);
index->add_item(index->get_n_items(), &column_arary_data_float_data[column_array_offsets[current_row - 1]]);
}
else if (const auto & column_tuple = typeid_cast<const ColumnTuple *>(column_cut.get()))
{
const auto & columns = column_tuple->getColumns();
const auto & column_tuple_columns = column_tuple->getColumns();
/// TODO check if calling index->add_item() directly on the block's tuples is faster than materializing everything
std::vector<std::vector<Float32>> data{column_tuple->size(), std::vector<Float32>()};
for (const auto & column : columns)
std::vector<std::vector<Float32>> data(column_tuple->size(), std::vector<Float32>());
for (const auto & column : column_tuple_columns)
{
const auto & pod_array = typeid_cast<const ColumnFloat32 *>(column.get())->getData();
for (size_t i = 0; i < pod_array.size(); ++i)
@ -363,7 +372,7 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */)
{
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Annoy indexes can only be created on columns of type Array(Float32) and Tuple(Float32)");
"Annoy indexes can only be created on columns of type Array(Float32) and Tuple(Float32[, Float32[, ...]])");
};
DataTypePtr data_type = index.sample_block.getDataTypes()[0];

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