mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge branch 'master' into mvcc_prototype
This commit is contained in:
commit
5a50ad9de3
10
.github/workflows/master.yml
vendored
10
.github/workflows/master.yml
vendored
@ -149,7 +149,6 @@ jobs:
|
||||
sudo rm -fr "$TEMP_PATH"
|
||||
SplitBuildSmokeTest:
|
||||
needs: [BuilderDebSplitted]
|
||||
if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }}
|
||||
runs-on: [self-hosted, style-checker]
|
||||
steps:
|
||||
- name: Set envs
|
||||
@ -316,7 +315,6 @@ jobs:
|
||||
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
|
||||
BuilderBinRelease:
|
||||
needs: [DockerHubPush]
|
||||
if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }}
|
||||
runs-on: [self-hosted, builder]
|
||||
steps:
|
||||
- name: Set envs
|
||||
@ -362,7 +360,6 @@ jobs:
|
||||
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
|
||||
BuilderBinGCC:
|
||||
needs: [DockerHubPush]
|
||||
if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }}
|
||||
runs-on: [self-hosted, builder]
|
||||
steps:
|
||||
- name: Set envs
|
||||
@ -636,7 +633,6 @@ jobs:
|
||||
##########################################################################################
|
||||
BuilderDebSplitted:
|
||||
needs: [DockerHubPush]
|
||||
if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }}
|
||||
runs-on: [self-hosted, builder]
|
||||
steps:
|
||||
- name: Set envs
|
||||
@ -682,7 +678,6 @@ jobs:
|
||||
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
|
||||
BuilderBinTidy:
|
||||
needs: [DockerHubPush]
|
||||
if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }}
|
||||
runs-on: [self-hosted, builder]
|
||||
steps:
|
||||
- name: Set envs
|
||||
@ -728,7 +723,6 @@ jobs:
|
||||
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
|
||||
BuilderBinDarwin:
|
||||
needs: [DockerHubPush]
|
||||
if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }}
|
||||
runs-on: [self-hosted, builder]
|
||||
steps:
|
||||
- name: Set envs
|
||||
@ -774,7 +768,6 @@ jobs:
|
||||
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
|
||||
BuilderBinAarch64:
|
||||
needs: [DockerHubPush]
|
||||
if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }}
|
||||
runs-on: [self-hosted, builder]
|
||||
steps:
|
||||
- name: Set envs
|
||||
@ -820,7 +813,6 @@ jobs:
|
||||
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
|
||||
BuilderBinFreeBSD:
|
||||
needs: [DockerHubPush]
|
||||
if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }}
|
||||
runs-on: [self-hosted, builder]
|
||||
steps:
|
||||
- name: Set envs
|
||||
@ -866,7 +858,6 @@ jobs:
|
||||
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
|
||||
BuilderBinDarwinAarch64:
|
||||
needs: [DockerHubPush]
|
||||
if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }}
|
||||
runs-on: [self-hosted, builder]
|
||||
steps:
|
||||
- name: Set envs
|
||||
@ -912,7 +903,6 @@ jobs:
|
||||
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
|
||||
BuilderBinPPC64:
|
||||
needs: [DockerHubPush]
|
||||
if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }}
|
||||
runs-on: [self-hosted, builder]
|
||||
steps:
|
||||
- name: Set envs
|
||||
|
@ -1,9 +1,12 @@
|
||||
if (APPLE OR NOT ARCH_AMD64 OR SANITIZE STREQUAL "undefined")
|
||||
set (ENABLE_EMBEDDED_COMPILER_DEFAULT OFF)
|
||||
# During cross-compilation in our CI we have to use llvm-tblgen and other building tools
|
||||
# tools to be build for host architecture and everything else for target architecture (e.g. AArch64)
|
||||
# Possible workaround is to use llvm-tblgen from some package...
|
||||
# But lets just enable LLVM for native builds
|
||||
if (CMAKE_CROSSCOMPILING OR SANITIZE STREQUAL "undefined")
|
||||
set (ENABLE_EMBEDDED_COMPILER_DEFAULT OFF)
|
||||
else()
|
||||
set (ENABLE_EMBEDDED_COMPILER_DEFAULT ON)
|
||||
set (ENABLE_EMBEDDED_COMPILER_DEFAULT ON)
|
||||
endif()
|
||||
|
||||
option (ENABLE_EMBEDDED_COMPILER "Enable support for 'compile_expressions' option for query execution" ${ENABLE_EMBEDDED_COMPILER_DEFAULT})
|
||||
|
||||
if (NOT ENABLE_EMBEDDED_COMPILER)
|
||||
|
@ -1378,7 +1378,7 @@ $REF_SHA $SHA_TO_TEST $(numactl --hardware | sed -n 's/^available:[[:space:]]\+/
|
||||
EOF
|
||||
|
||||
# Also insert some data about the check into the CI checks table.
|
||||
"${client[@]}" --query "INSERT INTO "'"'"gh-data"'"'".checks FORMAT TSVWithNamesAndTypes" \
|
||||
"${client[@]}" --query "INSERT INTO "'"'"default"'"'".checks FORMAT TSVWithNamesAndTypes" \
|
||||
< ci-checks.tsv
|
||||
|
||||
set -x
|
||||
|
@ -115,7 +115,7 @@ function run_tests()
|
||||
fi
|
||||
|
||||
set +e
|
||||
clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time \
|
||||
clickhouse-test -j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time \
|
||||
--skip 00168_parallel_processing_on_replicas "${ADDITIONAL_OPTIONS[@]}" \
|
||||
"$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt
|
||||
|
||||
|
@ -131,8 +131,23 @@ clickhouse-client -q "system flush logs" ||:
|
||||
|
||||
grep -Fa "Fatal" /var/log/clickhouse-server/clickhouse-server.log ||:
|
||||
pigz < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.gz &
|
||||
clickhouse-client -q "select * from system.query_log format TSVWithNamesAndTypes" | pigz > /test_output/query-log.tsv.gz &
|
||||
clickhouse-client -q "select * from system.query_thread_log format TSVWithNamesAndTypes" | pigz > /test_output/query-thread-log.tsv.gz &
|
||||
|
||||
# Compress tables.
|
||||
#
|
||||
# NOTE:
|
||||
# - that due to tests with s3 storage we cannot use /var/lib/clickhouse/data
|
||||
# directly
|
||||
# - even though ci auto-compress some files (but not *.tsv) it does this only
|
||||
# for files >64MB, we want this files to be compressed explicitly
|
||||
for table in query_log zookeeper_log trace_log transactions_info_log
|
||||
do
|
||||
clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.tsv.gz &
|
||||
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
|
||||
clickhouse-client --port 19000 -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.1.tsv.gz &
|
||||
clickhouse-client --port 29000 -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.2.tsv.gz &
|
||||
fi
|
||||
done
|
||||
wait ||:
|
||||
|
||||
# Also export trace log in flamegraph-friendly format.
|
||||
for trace_type in CPU Memory Real
|
||||
@ -161,14 +176,6 @@ fi
|
||||
|
||||
tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||:
|
||||
|
||||
# Replace the engine with Ordinary to avoid extra symlinks stuff in artifacts.
|
||||
# (so that clickhouse-local --path can read it w/o extra care).
|
||||
sed -i -e "s/ATTACH DATABASE _ UUID '[^']*'/ATTACH DATABASE system/" -e "s/Atomic/Ordinary/" /var/lib/clickhouse/metadata/system.sql
|
||||
for table in text_log query_log zookeeper_log trace_log transactions_info_log; do
|
||||
sed -i "s/ATTACH TABLE _ UUID '[^']*'/ATTACH TABLE $table/" /var/lib/clickhouse/metadata/system/${table}.sql
|
||||
tar -chf /test_output/${table}_dump.tar /var/lib/clickhouse/metadata/system.sql /var/lib/clickhouse/metadata/system/${table}.sql /var/lib/clickhouse/data/system/${table} ||:
|
||||
done
|
||||
|
||||
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
|
||||
grep -Fa "Fatal" /var/log/clickhouse-server/clickhouse-server1.log ||:
|
||||
grep -Fa "Fatal" /var/log/clickhouse-server/clickhouse-server2.log ||:
|
||||
@ -179,8 +186,6 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]
|
||||
rm /var/log/clickhouse-server/clickhouse-server2.log
|
||||
mv /var/log/clickhouse-server/stderr1.log /test_output/ ||:
|
||||
mv /var/log/clickhouse-server/stderr2.log /test_output/ ||:
|
||||
tar -chf /test_output/zookeeper_log_dump1.tar /var/lib/clickhouse1/data/system/zookeeper_log ||:
|
||||
tar -chf /test_output/zookeeper_log_dump2.tar /var/lib/clickhouse2/data/system/zookeeper_log ||:
|
||||
tar -chf /test_output/coordination1.tar /var/lib/clickhouse1/coordination ||:
|
||||
tar -chf /test_output/coordination2.tar /var/lib/clickhouse2/coordination ||:
|
||||
fi
|
||||
|
@ -0,0 +1,48 @@
|
||||
---
|
||||
toc_priority: 108
|
||||
---
|
||||
|
||||
# groupArraySorted {#groupArraySorted}
|
||||
|
||||
Returns an array with the first N items in ascending order.
|
||||
|
||||
``` sql
|
||||
groupArraySorted(N)(column)
|
||||
```
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `N` – The number of elements to return.
|
||||
|
||||
If the parameter is omitted, default value 10 is used.
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `column` – The value.
|
||||
- `expr` — Optional. The field or expresion to sort by. If not set values are sorted by themselves.
|
||||
|
||||
**Example**
|
||||
|
||||
Gets the first 10 numbers:
|
||||
|
||||
``` sql
|
||||
SELECT groupArraySorted(10)(number) FROM numbers(100)
|
||||
```
|
||||
|
||||
``` text
|
||||
┌─groupArraySorted(10)(number)─┐
|
||||
│ [0,1,2,3,4,5,6,7,8,9] │
|
||||
└──────────────────────────────┘
|
||||
```
|
||||
|
||||
Or the last 10:
|
||||
|
||||
``` sql
|
||||
SELECT groupArraySorted(10)(number, -number) FROM numbers(100)
|
||||
```
|
||||
|
||||
``` text
|
||||
┌─groupArraySorted(10)(number, negate(number))─┐
|
||||
│ [99,98,97,96,95,94,93,92,91,90] │
|
||||
└──────────────────────────────────────────────┘
|
||||
```
|
@ -35,6 +35,7 @@ ClickHouse-specific aggregate functions:
|
||||
- [groupArrayInsertAt](../../../sql-reference/aggregate-functions/reference/grouparrayinsertat.md)
|
||||
- [groupArrayMovingAvg](../../../sql-reference/aggregate-functions/reference/grouparraymovingavg.md)
|
||||
- [groupArrayMovingSum](../../../sql-reference/aggregate-functions/reference/grouparraymovingsum.md)
|
||||
- [groupArraySorted](../../../sql-reference/aggregate-functions/reference/grouparraysorted.md)
|
||||
- [groupBitAnd](../../../sql-reference/aggregate-functions/reference/groupbitand.md)
|
||||
- [groupBitOr](../../../sql-reference/aggregate-functions/reference/groupbitor.md)
|
||||
- [groupBitXor](../../../sql-reference/aggregate-functions/reference/groupbitxor.md)
|
||||
|
@ -8,7 +8,7 @@ toc_title: "版本折叠MergeTree"
|
||||
这个引擎:
|
||||
|
||||
- 允许快速写入不断变化的对象状态。
|
||||
- 删除后台中的旧对象状态。 这显着降低了存储体积。
|
||||
- 删除后台中的旧对象状态。 这显著降低了存储体积。
|
||||
|
||||
请参阅部分 [崩溃](#table_engines_versionedcollapsingmergetree) 有关详细信息。
|
||||
|
||||
|
@ -184,6 +184,11 @@ void LocalServer::tryInitPath()
|
||||
if (path.back() != '/')
|
||||
path += '/';
|
||||
|
||||
fs::create_directories(fs::path(path) / "user_defined/");
|
||||
fs::create_directories(fs::path(path) / "data/");
|
||||
fs::create_directories(fs::path(path) / "metadata/");
|
||||
fs::create_directories(fs::path(path) / "metadata_dropped/");
|
||||
|
||||
global_context->setPath(path);
|
||||
|
||||
global_context->setTemporaryStorage(path + "tmp");
|
||||
@ -565,7 +570,6 @@ void LocalServer::processConfig()
|
||||
/// Lock path directory before read
|
||||
status.emplace(fs::path(path) / "status", StatusFile::write_full_info);
|
||||
|
||||
fs::create_directories(fs::path(path) / "user_defined/");
|
||||
LOG_DEBUG(log, "Loading user defined objects from {}", path);
|
||||
Poco::File(path + "user_defined/").createDirectories();
|
||||
UserDefinedSQLObjectsLoader::instance().loadObjects(global_context);
|
||||
@ -573,9 +577,6 @@ void LocalServer::processConfig()
|
||||
LOG_DEBUG(log, "Loaded user defined objects.");
|
||||
|
||||
LOG_DEBUG(log, "Loading metadata from {}", path);
|
||||
fs::create_directories(fs::path(path) / "data/");
|
||||
fs::create_directories(fs::path(path) / "metadata/");
|
||||
|
||||
loadMetadataSystem(global_context);
|
||||
attachSystemTablesLocal(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE));
|
||||
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA));
|
||||
|
147
src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp
Normal file
147
src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp
Normal file
@ -0,0 +1,147 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/AggregateFunctionGroupArraySorted.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
|
||||
|
||||
static inline constexpr UInt64 GROUP_SORTED_ARRAY_MAX_SIZE = 0xFFFFFF;
|
||||
static inline constexpr UInt64 GROUP_SORTED_ARRAY_DEFAULT_THRESHOLD = 10;
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct Settings;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
template <typename T, bool expr_sorted, typename TColumnB, bool is_plain_b>
|
||||
class AggregateFunctionGroupArraySortedNumeric : public AggregateFunctionGroupArraySorted<T, false, expr_sorted, TColumnB, is_plain_b>
|
||||
{
|
||||
using AggregateFunctionGroupArraySorted<T, false, expr_sorted, TColumnB, is_plain_b>::AggregateFunctionGroupArraySorted;
|
||||
};
|
||||
|
||||
template <typename T, bool expr_sorted, typename TColumnB, bool is_plain_b>
|
||||
class AggregateFunctionGroupArraySortedFieldType
|
||||
: public AggregateFunctionGroupArraySorted<typename T::FieldType, false, expr_sorted, TColumnB, is_plain_b>
|
||||
{
|
||||
using AggregateFunctionGroupArraySorted<typename T::FieldType, false, expr_sorted, TColumnB, is_plain_b>::
|
||||
AggregateFunctionGroupArraySorted;
|
||||
DataTypePtr getReturnType() const override { return std::make_shared<DataTypeArray>(std::make_shared<T>()); }
|
||||
};
|
||||
|
||||
template <template <typename, bool, typename, bool> class AggregateFunctionTemplate, typename TColumnA, bool expr_sorted, typename TColumnB, bool is_plain_b, typename... TArgs>
|
||||
AggregateFunctionPtr
|
||||
createAggregateFunctionGroupArraySortedTypedFinal(TArgs && ... args)
|
||||
{
|
||||
return AggregateFunctionPtr(new AggregateFunctionTemplate<TColumnA, expr_sorted, TColumnB, is_plain_b>(std::forward<TArgs>(args)...));
|
||||
}
|
||||
|
||||
template <bool expr_sorted = false, typename TColumnB = UInt64, bool is_plain_b = false>
|
||||
AggregateFunctionPtr
|
||||
createAggregateFunctionGroupArraySortedTyped(const DataTypes & argument_types, const Array & params, UInt64 threshold)
|
||||
{
|
||||
#define DISPATCH(A, C, B) \
|
||||
if (which.idx == TypeIndex::A) \
|
||||
return createAggregateFunctionGroupArraySortedTypedFinal<C, B, expr_sorted, TColumnB, is_plain_b>(threshold, argument_types, params);
|
||||
#define DISPATCH_NUMERIC(A) DISPATCH(A, AggregateFunctionGroupArraySortedNumeric, A)
|
||||
WhichDataType which(argument_types[0]);
|
||||
FOR_NUMERIC_TYPES(DISPATCH_NUMERIC)
|
||||
DISPATCH(Enum8, AggregateFunctionGroupArraySortedNumeric, Int8)
|
||||
DISPATCH(Enum16, AggregateFunctionGroupArraySortedNumeric, Int16)
|
||||
DISPATCH(Date, AggregateFunctionGroupArraySortedFieldType, DataTypeDate)
|
||||
DISPATCH(DateTime, AggregateFunctionGroupArraySortedFieldType, DataTypeDateTime)
|
||||
#undef DISPATCH
|
||||
#undef DISPATCH_NUMERIC
|
||||
|
||||
if (argument_types[0]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
|
||||
{
|
||||
return AggregateFunctionPtr(new AggregateFunctionGroupArraySorted<StringRef, true, expr_sorted, TColumnB, is_plain_b>(
|
||||
threshold, argument_types, params));
|
||||
}
|
||||
else
|
||||
{
|
||||
return AggregateFunctionPtr(new AggregateFunctionGroupArraySorted<StringRef, false, expr_sorted, TColumnB, is_plain_b>(
|
||||
threshold, argument_types, params));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
AggregateFunctionPtr createAggregateFunctionGroupArraySorted(
|
||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
||||
{
|
||||
UInt64 threshold = GROUP_SORTED_ARRAY_DEFAULT_THRESHOLD;
|
||||
|
||||
if (params.size() == 1)
|
||||
{
|
||||
UInt64 k = applyVisitor(FieldVisitorConvertToNumber<UInt64>(), params[0]);
|
||||
|
||||
if (k > GROUP_SORTED_ARRAY_MAX_SIZE)
|
||||
throw Exception(
|
||||
"Too large parameter(s) for aggregate function " + name + ". Maximum: " + toString(GROUP_SORTED_ARRAY_MAX_SIZE),
|
||||
ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
if (k == 0)
|
||||
throw Exception("Parameter 0 is illegal for aggregate function " + name, ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
threshold = k;
|
||||
}
|
||||
else if (!params.empty())
|
||||
{
|
||||
throw Exception("Aggregate function " + name + " only supports 1 parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
|
||||
if (argument_types.size() == 2)
|
||||
{
|
||||
if (isNumber(argument_types[1]))
|
||||
{
|
||||
#define DISPATCH2(A, B) \
|
||||
if (which.idx == TypeIndex::A) \
|
||||
return createAggregateFunctionGroupArraySortedTyped<true, B>(argument_types, params, threshold);
|
||||
#define DISPATCH(A) DISPATCH2(A, A)
|
||||
WhichDataType which(argument_types[1]);
|
||||
FOR_NUMERIC_TYPES(DISPATCH)
|
||||
DISPATCH2(Enum8, Int8)
|
||||
DISPATCH2(Enum16, Int16)
|
||||
#undef DISPATCH
|
||||
#undef DISPATCH2
|
||||
throw Exception("Invalid parameter type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
else if (argument_types[1]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
|
||||
{
|
||||
return createAggregateFunctionGroupArraySortedTyped<true, StringRef, true>(argument_types, params, threshold);
|
||||
}
|
||||
else
|
||||
{
|
||||
return createAggregateFunctionGroupArraySortedTyped<true, StringRef, false>(argument_types, params, threshold);
|
||||
}
|
||||
}
|
||||
else if (argument_types.size() == 1)
|
||||
{
|
||||
return createAggregateFunctionGroupArraySortedTyped<>(argument_types, params, threshold);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
"Aggregate function " + name + " requires one or two parameters.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void registerAggregateFunctionGroupArraySorted(AggregateFunctionFactory & factory)
|
||||
{
|
||||
AggregateFunctionProperties properties = {.returns_default_when_only_null = false, .is_order_dependent = true};
|
||||
factory.registerFunction("groupArraySorted", {createAggregateFunctionGroupArraySorted, properties});
|
||||
}
|
||||
}
|
310
src/AggregateFunctions/AggregateFunctionGroupArraySorted.h
Normal file
310
src/AggregateFunctions/AggregateFunctionGroupArraySorted.h
Normal file
@ -0,0 +1,310 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionGroupArraySortedData.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
template <typename TColumn, bool is_plain>
|
||||
inline TColumn readItem(const IColumn * column, Arena * arena, size_t row)
|
||||
{
|
||||
if constexpr (std::is_same_v<TColumn, StringRef>)
|
||||
{
|
||||
if constexpr (is_plain)
|
||||
{
|
||||
StringRef str = column->getDataAt(row);
|
||||
auto ptr = arena->alloc(str.size);
|
||||
std::copy(str.data, str.data + str.size, ptr);
|
||||
return StringRef(ptr, str.size);
|
||||
}
|
||||
else
|
||||
{
|
||||
const char * begin = nullptr;
|
||||
return column->serializeValueIntoArena(row, *arena, begin);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
if constexpr (std::is_same_v<TColumn, UInt64>)
|
||||
return column->getUInt(row);
|
||||
else
|
||||
return column->getInt(row);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename TColumn, typename TFilter = void>
|
||||
size_t
|
||||
getFirstNElements_low_threshold(const TColumn * data, int num_elements, int threshold, size_t * results, const TFilter * filter = nullptr)
|
||||
{
|
||||
for (int i = 0; i < threshold; i++)
|
||||
{
|
||||
results[i] = 0;
|
||||
}
|
||||
|
||||
threshold = std::min(num_elements, threshold);
|
||||
int current_max = 0;
|
||||
int cur;
|
||||
int z;
|
||||
for (int i = 0; i < num_elements; i++)
|
||||
{
|
||||
if constexpr (!std::is_same_v<TFilter, void>)
|
||||
{
|
||||
if (filter[i] == 0)
|
||||
continue;
|
||||
}
|
||||
|
||||
//Starting from the highest values and we look for the immediately lower than the given one
|
||||
for (cur = current_max; cur > 0; cur--)
|
||||
{
|
||||
if (data[i] > data[results[cur - 1]])
|
||||
break;
|
||||
}
|
||||
|
||||
if (cur < threshold)
|
||||
{
|
||||
//Move all the higher values 1 position to the right
|
||||
for (z = std::min(threshold - 1, current_max); z > cur; z--)
|
||||
results[z] = results[z - 1];
|
||||
|
||||
if (current_max < threshold)
|
||||
++current_max;
|
||||
|
||||
//insert element into the given position
|
||||
results[cur] = i;
|
||||
}
|
||||
}
|
||||
|
||||
return current_max;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
struct SortableItem
|
||||
{
|
||||
T a;
|
||||
size_t b;
|
||||
bool operator<(const SortableItem & other) const { return (this->a < other.a); }
|
||||
};
|
||||
|
||||
template <typename TColumn, typename TFilter = void>
|
||||
size_t getFirstNElements_high_threshold(
|
||||
const TColumn * data, size_t num_elements, size_t threshold, size_t * results, const TFilter * filter = nullptr)
|
||||
{
|
||||
std::vector<SortableItem<TColumn>> dataIndexed(num_elements);
|
||||
size_t num_elements_filtered = 0;
|
||||
|
||||
for (size_t i = 0; i < num_elements; i++)
|
||||
{
|
||||
if constexpr (!std::is_same_v<TFilter, void>)
|
||||
{
|
||||
if (filter[i] == 0)
|
||||
continue;
|
||||
}
|
||||
|
||||
dataIndexed.data()[num_elements_filtered].a = data[i];
|
||||
dataIndexed.data()[num_elements_filtered].b = i;
|
||||
num_elements_filtered++;
|
||||
}
|
||||
|
||||
threshold = std::min(num_elements_filtered, threshold);
|
||||
|
||||
std::nth_element(dataIndexed.data(), dataIndexed.data() + threshold, dataIndexed.data() + num_elements_filtered);
|
||||
std::sort(dataIndexed.data(), dataIndexed.data() + threshold);
|
||||
|
||||
for (size_t i = 0; i < threshold; i++)
|
||||
{
|
||||
results[i] = dataIndexed[i].b;
|
||||
}
|
||||
|
||||
return threshold;
|
||||
}
|
||||
|
||||
static const size_t THRESHOLD_MAX_CUSTOM_FUNCTION = 1000;
|
||||
|
||||
template <typename TColumn>
|
||||
size_t getFirstNElements(const TColumn * data, size_t num_elements, size_t threshold, size_t * results, const UInt8 * filter = nullptr)
|
||||
{
|
||||
if (threshold < THRESHOLD_MAX_CUSTOM_FUNCTION)
|
||||
{
|
||||
if (filter != nullptr)
|
||||
return getFirstNElements_low_threshold(data, num_elements, threshold, results, filter);
|
||||
else
|
||||
return getFirstNElements_low_threshold(data, num_elements, threshold, results);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (filter != nullptr)
|
||||
return getFirstNElements_high_threshold(data, num_elements, threshold, results, filter);
|
||||
else
|
||||
return getFirstNElements_high_threshold(data, num_elements, threshold, results);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename TColumnA, bool is_plain_a, bool use_column_b, typename TColumnB, bool is_plain_b>
|
||||
class AggregateFunctionGroupArraySorted : public IAggregateFunctionDataHelper<
|
||||
AggregateFunctionGroupArraySortedData<TColumnA, use_column_b, TColumnB>,
|
||||
AggregateFunctionGroupArraySorted<TColumnA, is_plain_a, use_column_b, TColumnB, is_plain_b>>
|
||||
{
|
||||
protected:
|
||||
using State = AggregateFunctionGroupArraySortedData<TColumnA, use_column_b, TColumnB>;
|
||||
using Base = IAggregateFunctionDataHelper<
|
||||
AggregateFunctionGroupArraySortedData<TColumnA, use_column_b, TColumnB>,
|
||||
AggregateFunctionGroupArraySorted>;
|
||||
|
||||
UInt64 threshold;
|
||||
DataTypePtr & input_data_type;
|
||||
mutable std::mutex mutex;
|
||||
|
||||
static void deserializeAndInsert(StringRef str, IColumn & data_to);
|
||||
|
||||
public:
|
||||
AggregateFunctionGroupArraySorted(UInt64 threshold_, const DataTypes & argument_types_, const Array & params)
|
||||
: IAggregateFunctionDataHelper<
|
||||
AggregateFunctionGroupArraySortedData<TColumnA, use_column_b, TColumnB>,
|
||||
AggregateFunctionGroupArraySorted>(argument_types_, params)
|
||||
, threshold(threshold_)
|
||||
, input_data_type(this->argument_types[0])
|
||||
{
|
||||
}
|
||||
|
||||
void create(AggregateDataPtr place) const override
|
||||
{
|
||||
Base::create(place);
|
||||
this->data(place).threshold = threshold;
|
||||
}
|
||||
|
||||
String getName() const override { return "groupArraySorted"; }
|
||||
|
||||
DataTypePtr getReturnType() const override { return std::make_shared<DataTypeArray>(input_data_type); }
|
||||
|
||||
bool allocatesMemoryInArena() const override
|
||||
{
|
||||
if constexpr (std::is_same_v<TColumnA, StringRef>)
|
||||
return true;
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
|
||||
{
|
||||
State & data = this->data(place);
|
||||
if constexpr (use_column_b)
|
||||
{
|
||||
data.add(
|
||||
readItem<TColumnA, is_plain_a>(columns[0], arena, row_num), readItem<TColumnB, is_plain_b>(columns[1], arena, row_num));
|
||||
}
|
||||
else
|
||||
{
|
||||
data.add(readItem<TColumnA, is_plain_a>(columns[0], arena, row_num));
|
||||
}
|
||||
}
|
||||
|
||||
template <typename TColumn, bool is_plain, typename TFunc>
|
||||
void
|
||||
forFirstRows(size_t batch_size, const IColumn ** columns, size_t data_column, Arena * arena, ssize_t if_argument_pos, TFunc func) const
|
||||
{
|
||||
const TColumn * values = nullptr;
|
||||
std::unique_ptr<std::vector<TColumn>> values_vector;
|
||||
std::vector<size_t> best_rows(threshold);
|
||||
|
||||
if constexpr (std::is_same_v<TColumn, StringRef>)
|
||||
{
|
||||
values_vector.reset(new std::vector<TColumn>(batch_size));
|
||||
for (size_t i = 0; i < batch_size; i++)
|
||||
(*values_vector)[i] = readItem<TColumn, is_plain>(columns[data_column], arena, i);
|
||||
values = (*values_vector).data();
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & column = assert_cast<const ColumnVector<TColumn> &>(*columns[data_column]);
|
||||
values = column.getData().data();
|
||||
}
|
||||
|
||||
const UInt8 * filter = nullptr;
|
||||
StringRef refFilter;
|
||||
|
||||
if (if_argument_pos >= 0)
|
||||
{
|
||||
refFilter = columns[if_argument_pos]->getRawData();
|
||||
filter = reinterpret_cast<const UInt8 *>(refFilter.data);
|
||||
}
|
||||
|
||||
size_t num_elements = getFirstNElements(values, batch_size, threshold, best_rows.data(), filter);
|
||||
for (size_t i = 0; i < num_elements; i++)
|
||||
{
|
||||
func(best_rows[i], values);
|
||||
}
|
||||
}
|
||||
|
||||
void addBatchSinglePlace(
|
||||
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos) const override
|
||||
{
|
||||
State & data = this->data(place);
|
||||
|
||||
if constexpr (use_column_b)
|
||||
{
|
||||
forFirstRows<TColumnB, is_plain_b>(
|
||||
batch_size, columns, 1, arena, if_argument_pos, [columns, &arena, &data](size_t row, const TColumnB * values)
|
||||
{
|
||||
data.add(readItem<TColumnA, is_plain_a>(columns[0], arena, row), values[row]);
|
||||
});
|
||||
}
|
||||
else
|
||||
{
|
||||
forFirstRows<TColumnA, is_plain_a>(
|
||||
batch_size, columns, 0, arena, if_argument_pos, [&data](size_t row, const TColumnA * values)
|
||||
{
|
||||
data.add(values[row]);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
|
||||
{
|
||||
this->data(place).merge(this->data(rhs));
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
|
||||
{
|
||||
this->data(place).serialize(buf);
|
||||
}
|
||||
|
||||
void
|
||||
deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
|
||||
{
|
||||
this->data(place).deserialize(buf, arena);
|
||||
}
|
||||
|
||||
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena * /*arena*/) const override
|
||||
{
|
||||
ColumnArray & arr_to = assert_cast<ColumnArray &>(to);
|
||||
ColumnArray::Offsets & offsets_to = arr_to.getOffsets();
|
||||
|
||||
auto & values = this->data(place).values;
|
||||
offsets_to.push_back(offsets_to.back() + values.size());
|
||||
|
||||
IColumn & data_to = arr_to.getData();
|
||||
for (auto value : values)
|
||||
{
|
||||
if constexpr (std::is_same_v<TColumnA, StringRef>)
|
||||
{
|
||||
auto str = State::itemValue(value);
|
||||
if constexpr (is_plain_a)
|
||||
{
|
||||
data_to.insertData(str.data, str.size);
|
||||
}
|
||||
else
|
||||
{
|
||||
data_to.deserializeAndInsertFromArena(str.data);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
data_to.insert(State::itemValue(value));
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
162
src/AggregateFunctions/AggregateFunctionGroupArraySortedData.h
Normal file
162
src/AggregateFunctions/AggregateFunctionGroupArraySortedData.h
Normal file
@ -0,0 +1,162 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/VarInt.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
||||
static inline constexpr UInt64 GROUP_SORTED_DEFAULT_THRESHOLD = 0xFFFFFF;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
template <typename T>
|
||||
static void writeOneItem(WriteBuffer & buf, T item)
|
||||
{
|
||||
if constexpr (std::numeric_limits<T>::is_signed)
|
||||
{
|
||||
writeVarInt(item, buf);
|
||||
}
|
||||
else
|
||||
{
|
||||
writeVarUInt(item, buf);
|
||||
}
|
||||
}
|
||||
|
||||
static void writeOneItem(WriteBuffer & buf, const StringRef & item)
|
||||
{
|
||||
writeBinary(item, buf);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static void readOneItem(ReadBuffer & buf, Arena * /*arena*/, T & item)
|
||||
{
|
||||
if constexpr (std::numeric_limits<T>::is_signed)
|
||||
{
|
||||
DB::Int64 val;
|
||||
readVarT(val, buf);
|
||||
item = val;
|
||||
}
|
||||
else
|
||||
{
|
||||
DB::UInt64 val;
|
||||
readVarT(val, buf);
|
||||
item = val;
|
||||
}
|
||||
}
|
||||
|
||||
static void readOneItem(ReadBuffer & buf, Arena * arena, StringRef & item)
|
||||
{
|
||||
item = readStringBinaryInto(*arena, buf);
|
||||
}
|
||||
|
||||
template <typename Storage>
|
||||
struct AggregateFunctionGroupArraySortedDataBase
|
||||
{
|
||||
typedef typename Storage::value_type ValueType;
|
||||
AggregateFunctionGroupArraySortedDataBase(UInt64 threshold_ = GROUP_SORTED_DEFAULT_THRESHOLD) : threshold(threshold_) { }
|
||||
|
||||
virtual ~AggregateFunctionGroupArraySortedDataBase() { }
|
||||
inline void narrowDown()
|
||||
{
|
||||
while (values.size() > threshold)
|
||||
values.erase(--values.end());
|
||||
}
|
||||
|
||||
void merge(const AggregateFunctionGroupArraySortedDataBase & other)
|
||||
{
|
||||
values.merge(Storage(other.values));
|
||||
narrowDown();
|
||||
}
|
||||
|
||||
void serialize(WriteBuffer & buf) const
|
||||
{
|
||||
writeOneItem(buf, UInt64(values.size()));
|
||||
for (auto value : values)
|
||||
{
|
||||
serializeItem(buf, value);
|
||||
}
|
||||
}
|
||||
|
||||
virtual void serializeItem(WriteBuffer & buf, ValueType & val) const = 0;
|
||||
virtual ValueType deserializeItem(ReadBuffer & buf, Arena * arena) const = 0;
|
||||
|
||||
void deserialize(ReadBuffer & buf, Arena * arena)
|
||||
{
|
||||
values.clear();
|
||||
UInt64 length;
|
||||
readOneItem(buf, nullptr, length);
|
||||
|
||||
while (length--)
|
||||
{
|
||||
values.insert(deserializeItem(buf, arena));
|
||||
}
|
||||
|
||||
narrowDown();
|
||||
}
|
||||
|
||||
UInt64 threshold;
|
||||
Storage values;
|
||||
};
|
||||
|
||||
template <typename T, bool expr_sorted, typename TIndex>
|
||||
struct AggregateFunctionGroupArraySortedData
|
||||
{
|
||||
};
|
||||
|
||||
template <typename T, typename TIndex>
|
||||
struct AggregateFunctionGroupArraySortedData<T, true, TIndex> : public AggregateFunctionGroupArraySortedDataBase<std::multimap<TIndex, T>>
|
||||
{
|
||||
using Base = AggregateFunctionGroupArraySortedDataBase<std::multimap<TIndex, T>>;
|
||||
using Base::Base;
|
||||
|
||||
void add(T item, TIndex weight)
|
||||
{
|
||||
Base::values.insert({weight, item});
|
||||
Base::narrowDown();
|
||||
}
|
||||
|
||||
void serializeItem(WriteBuffer & buf, typename Base::ValueType & value) const override
|
||||
{
|
||||
writeOneItem(buf, value.first);
|
||||
writeOneItem(buf, value.second);
|
||||
}
|
||||
|
||||
virtual typename Base::ValueType deserializeItem(ReadBuffer & buf, Arena * arena) const override
|
||||
{
|
||||
TIndex first;
|
||||
T second;
|
||||
readOneItem(buf, arena, first);
|
||||
readOneItem(buf, arena, second);
|
||||
|
||||
return {first, second};
|
||||
}
|
||||
|
||||
static T itemValue(typename Base::ValueType & value) { return value.second; }
|
||||
};
|
||||
|
||||
template <typename T, typename TIndex>
|
||||
struct AggregateFunctionGroupArraySortedData<T, false, TIndex> : public AggregateFunctionGroupArraySortedDataBase<std::multiset<T>>
|
||||
{
|
||||
using Base = AggregateFunctionGroupArraySortedDataBase<std::multiset<T>>;
|
||||
using Base::Base;
|
||||
|
||||
void add(T item)
|
||||
{
|
||||
Base::values.insert(item);
|
||||
Base::narrowDown();
|
||||
}
|
||||
|
||||
void serializeItem(WriteBuffer & buf, typename Base::ValueType & value) const override { writeOneItem(buf, value); }
|
||||
|
||||
typename Base::ValueType deserializeItem(ReadBuffer & buf, Arena * arena) const override
|
||||
{
|
||||
T value;
|
||||
readOneItem(buf, arena, value);
|
||||
return value;
|
||||
}
|
||||
|
||||
static T itemValue(typename Base::ValueType & value) { return value; }
|
||||
};
|
||||
}
|
@ -59,6 +59,7 @@ void registerAggregateFunctionNothing(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionExponentialMovingAverage(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionSparkbar(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionIntervalLengthSum(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionGroupArraySorted(AggregateFunctionFactory & factory);
|
||||
|
||||
class AggregateFunctionCombinatorFactory;
|
||||
void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &);
|
||||
@ -130,6 +131,7 @@ void registerAggregateFunctions()
|
||||
registerAggregateFunctionIntervalLengthSum(factory);
|
||||
registerAggregateFunctionExponentialMovingAverage(factory);
|
||||
registerAggregateFunctionSparkbar(factory);
|
||||
registerAggregateFunctionGroupArraySorted(factory);
|
||||
|
||||
registerWindowFunctions(factory);
|
||||
}
|
||||
|
@ -35,10 +35,10 @@ public:
|
||||
{}
|
||||
|
||||
// Format message with fmt::format, like the logging functions.
|
||||
template <typename ...Args>
|
||||
Exception(int code, const std::string & fmt, Args&&... args)
|
||||
: Exception(fmt::format(fmt::runtime(fmt), std::forward<Args>(args)...), code)
|
||||
{}
|
||||
template <typename... Args>
|
||||
Exception(int code, fmt::format_string<Args...> fmt, Args &&... args) : Exception(fmt::format(fmt, std::forward<Args>(args)...), code)
|
||||
{
|
||||
}
|
||||
|
||||
struct CreateFromPocoTag {};
|
||||
struct CreateFromSTDTag {};
|
||||
@ -52,10 +52,10 @@ public:
|
||||
const char * what() const throw() override { return message().data(); }
|
||||
|
||||
/// Add something to the existing message.
|
||||
template <typename ...Args>
|
||||
void addMessage(const std::string& format, Args&&... args)
|
||||
template <typename... Args>
|
||||
void addMessage(fmt::format_string<Args...> format, Args &&... args)
|
||||
{
|
||||
extendedMessage(fmt::format(fmt::runtime(format), std::forward<Args>(args)...));
|
||||
extendedMessage(fmt::format(format, std::forward<Args>(args)...));
|
||||
}
|
||||
|
||||
void addMessage(const std::string& message)
|
||||
@ -117,10 +117,10 @@ public:
|
||||
ParsingException(int code, const std::string & message);
|
||||
|
||||
// Format message with fmt::format, like the logging functions.
|
||||
template <typename ...Args>
|
||||
ParsingException(int code, const std::string & fmt, Args&&... args)
|
||||
: Exception(fmt::format(fmt::runtime(fmt), std::forward<Args>(args)...), code)
|
||||
{}
|
||||
template <typename... Args>
|
||||
ParsingException(int code, fmt::format_string<Args...> fmt, Args &&... args) : Exception(code, fmt, std::forward<Args>(args)...)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
std::string displayText() const
|
||||
|
@ -67,6 +67,9 @@ struct FixedHashTableCalculatedSize
|
||||
{
|
||||
size_t getSize(const Cell * buf, const typename Cell::State & state, size_t num_cells) const
|
||||
{
|
||||
if (!buf)
|
||||
return 0;
|
||||
|
||||
size_t res = 0;
|
||||
for (const Cell * end = buf + num_cells; buf != end; ++buf)
|
||||
if (!buf->isZero(state))
|
||||
@ -76,6 +79,9 @@ struct FixedHashTableCalculatedSize
|
||||
|
||||
bool isEmpty(const Cell * buf, const typename Cell::State & state, size_t num_cells) const
|
||||
{
|
||||
if (!buf)
|
||||
return true;
|
||||
|
||||
for (const Cell * end = buf + num_cells; buf != end; ++buf)
|
||||
if (!buf->isZero(state))
|
||||
return false;
|
||||
|
@ -94,6 +94,12 @@ public:
|
||||
|
||||
TwoLevelHashTable() = default;
|
||||
|
||||
explicit TwoLevelHashTable(size_t size_hint)
|
||||
{
|
||||
for (auto & impl : impls)
|
||||
impl.reserve(size_hint / NUM_BUCKETS);
|
||||
}
|
||||
|
||||
/// Copy the data from another (normal) hash table. It should have the same hash function.
|
||||
template <typename Source>
|
||||
explicit TwoLevelHashTable(const Source & src)
|
||||
|
@ -285,6 +285,9 @@
|
||||
\
|
||||
M(MainConfigLoads, "Number of times the main configuration was reloaded.") \
|
||||
\
|
||||
M(AggregationPreallocatedElementsInHashTables, "How many elements were preallocated in hash tables for aggregation.") \
|
||||
M(AggregationHashTablesInitializedAsTwoLevel, "How many hash tables were inited as two-level for aggregation.") \
|
||||
\
|
||||
M(MergeTreeMetadataCacheGet, "Number of rocksdb reads(used for merge tree metadata cache)") \
|
||||
M(MergeTreeMetadataCachePut, "Number of rocksdb puts(used for merge tree metadata cache)") \
|
||||
M(MergeTreeMetadataCacheDelete, "Number of rocksdb deletes(used for merge tree metadata cache)") \
|
||||
|
46
src/Common/RangeGenerator.h
Normal file
46
src/Common/RangeGenerator.h
Normal file
@ -0,0 +1,46 @@
|
||||
#pragma once
|
||||
|
||||
#include <optional>
|
||||
#include <cmath>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class RangeGenerator
|
||||
{
|
||||
public:
|
||||
explicit RangeGenerator(size_t total_size_, size_t range_step_, size_t range_start = 0)
|
||||
: from(range_start), range_step(range_step_), total_size(total_size_)
|
||||
{
|
||||
}
|
||||
|
||||
size_t totalRanges() const { return static_cast<size_t>(round(static_cast<float>(total_size - from) / range_step)); }
|
||||
|
||||
using Range = std::pair<size_t, size_t>;
|
||||
|
||||
// return upper exclusive range of values, i.e. [from_range, to_range>
|
||||
std::optional<Range> nextRange()
|
||||
{
|
||||
if (from >= total_size)
|
||||
{
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
auto to = from + range_step;
|
||||
if (to >= total_size)
|
||||
{
|
||||
to = total_size;
|
||||
}
|
||||
|
||||
Range range{from, to};
|
||||
from = to;
|
||||
return range;
|
||||
}
|
||||
|
||||
private:
|
||||
size_t from;
|
||||
size_t range_step;
|
||||
size_t total_size;
|
||||
};
|
||||
|
||||
}
|
178
src/Common/format.h
Normal file
178
src/Common/format.h
Normal file
@ -0,0 +1,178 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace Format
|
||||
{
|
||||
using IndexPositions = PODArrayWithStackMemory<UInt64, 64>;
|
||||
|
||||
static inline void parseNumber(const String & description, UInt64 l, UInt64 r, UInt64 & res, UInt64 argument_number)
|
||||
{
|
||||
res = 0;
|
||||
for (UInt64 pos = l; pos < r; ++pos)
|
||||
{
|
||||
if (!isNumericASCII(description[pos]))
|
||||
throw Exception("Not a number in curly braces at position " + std::to_string(pos), ErrorCodes::BAD_ARGUMENTS);
|
||||
res = res * 10 + description[pos] - '0';
|
||||
if (res >= argument_number)
|
||||
throw Exception(
|
||||
"Too big number for arguments, must be at most " + std::to_string(argument_number - 1), ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
}
|
||||
|
||||
static inline void init(
|
||||
const String & pattern,
|
||||
size_t argument_number,
|
||||
const std::vector<std::optional<String>> & constant_strings,
|
||||
IndexPositions & index_positions,
|
||||
std::vector<String> & substrings)
|
||||
{
|
||||
/// Is current position after open curly brace.
|
||||
bool is_open_curly = false;
|
||||
/// The position of last open token.
|
||||
size_t last_open = -1;
|
||||
|
||||
/// Is formatting in a plain {} token.
|
||||
std::optional<bool> is_plain_numbering;
|
||||
UInt64 index_if_plain = 0;
|
||||
|
||||
/// Left position of adding substrings, just to the closed brace position or the start of the string.
|
||||
/// Invariant --- the start of substring is in this position.
|
||||
size_t start_pos = 0;
|
||||
|
||||
/// A flag to decide whether we should glue the constant strings.
|
||||
bool glue_to_next = false;
|
||||
|
||||
/// Handling double braces (escaping).
|
||||
auto double_brace_removal = [](String & str)
|
||||
{
|
||||
size_t i = 0;
|
||||
bool should_delete = true;
|
||||
str.erase(
|
||||
std::remove_if(
|
||||
str.begin(),
|
||||
str.end(),
|
||||
[&i, &should_delete, &str](char)
|
||||
{
|
||||
bool is_double_brace = (str[i] == '{' && str[i + 1] == '{') || (str[i] == '}' && str[i + 1] == '}');
|
||||
++i;
|
||||
if (is_double_brace && should_delete)
|
||||
{
|
||||
should_delete = false;
|
||||
return true;
|
||||
}
|
||||
should_delete = true;
|
||||
return false;
|
||||
}),
|
||||
str.end());
|
||||
};
|
||||
|
||||
index_positions.emplace_back();
|
||||
|
||||
for (size_t i = 0; i < pattern.size(); ++i)
|
||||
{
|
||||
if (pattern[i] == '{')
|
||||
{
|
||||
/// Escaping handling
|
||||
/// It is safe to access because of null termination
|
||||
if (pattern[i + 1] == '{')
|
||||
{
|
||||
++i;
|
||||
continue;
|
||||
}
|
||||
|
||||
if (is_open_curly)
|
||||
throw Exception("Two open curly braces without close one at position " + std::to_string(i), ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
String to_add = String(pattern.data() + start_pos, i - start_pos);
|
||||
double_brace_removal(to_add);
|
||||
if (!glue_to_next)
|
||||
substrings.emplace_back(to_add);
|
||||
else
|
||||
substrings.back() += to_add;
|
||||
|
||||
glue_to_next = false;
|
||||
|
||||
is_open_curly = true;
|
||||
last_open = i + 1;
|
||||
}
|
||||
else if (pattern[i] == '}')
|
||||
{
|
||||
if (pattern[i + 1] == '}')
|
||||
{
|
||||
++i;
|
||||
continue;
|
||||
}
|
||||
|
||||
if (!is_open_curly)
|
||||
throw Exception("Closed curly brace without open one at position " + std::to_string(i), ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
is_open_curly = false;
|
||||
|
||||
if (last_open == i)
|
||||
{
|
||||
if (is_plain_numbering && !*is_plain_numbering)
|
||||
throw Exception(
|
||||
"Cannot switch from automatic field numbering to manual field specification", ErrorCodes::BAD_ARGUMENTS);
|
||||
is_plain_numbering = true;
|
||||
if (index_if_plain >= argument_number)
|
||||
throw Exception("Argument is too big for formatting", ErrorCodes::BAD_ARGUMENTS);
|
||||
index_positions.back() = index_if_plain++;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (is_plain_numbering && *is_plain_numbering)
|
||||
throw Exception(
|
||||
"Cannot switch from automatic field numbering to manual field specification", ErrorCodes::BAD_ARGUMENTS);
|
||||
is_plain_numbering = false;
|
||||
|
||||
UInt64 arg;
|
||||
parseNumber(pattern, last_open, i, arg, argument_number);
|
||||
|
||||
if (arg >= argument_number)
|
||||
throw Exception(
|
||||
"Argument is too big for formatting. Note that indexing starts from zero", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
index_positions.back() = arg;
|
||||
}
|
||||
|
||||
if (!constant_strings.empty() && constant_strings[index_positions.back()])
|
||||
{
|
||||
/// The next string should be glued to last `A {} C`.format('B') -> `A B C`.
|
||||
glue_to_next = true;
|
||||
substrings.back() += *constant_strings[index_positions.back()];
|
||||
}
|
||||
else
|
||||
index_positions.emplace_back(); /// Otherwise we commit arg number and proceed.
|
||||
|
||||
start_pos = i + 1;
|
||||
}
|
||||
}
|
||||
|
||||
if (is_open_curly)
|
||||
throw Exception("Last open curly brace is not closed", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
String to_add = String(pattern.data() + start_pos, pattern.size() - start_pos);
|
||||
double_brace_removal(to_add);
|
||||
|
||||
if (!glue_to_next)
|
||||
substrings.emplace_back(to_add);
|
||||
else
|
||||
substrings.back() += to_add;
|
||||
|
||||
index_positions.pop_back();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -500,6 +500,10 @@ class IColumn;
|
||||
M(Bool, optimize_rewrite_sum_if_to_count_if, true, "Rewrite sumIf() and sum(if()) function countIf() function when logically equivalent", 0) \
|
||||
M(UInt64, insert_shard_id, 0, "If non zero, when insert into a distributed table, the data will be inserted into the shard `insert_shard_id` synchronously. Possible values range from 1 to `shards_number` of corresponding distributed table", 0) \
|
||||
\
|
||||
M(Bool, collect_hash_table_stats_during_aggregation, true, "Enable collecting hash table statistics to optimize memory allocation", 0) \
|
||||
M(UInt64, max_entries_for_hash_table_stats, 10'000, "How many entries hash table statistics collected during aggregation is allowed to have", 0) \
|
||||
M(UInt64, max_size_to_preallocate_for_aggregation, 10'000'000, "For how many elements it is allowed to preallocate space in all hash tables in total before aggregation", 0) \
|
||||
\
|
||||
/** Experimental feature for moving data between shards. */ \
|
||||
\
|
||||
M(Bool, allow_experimental_query_deduplication, false, "Experimental data deduplication for SELECT queries based on part UUIDs", 0) \
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/ApplyWithSubqueryVisitor.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
@ -55,6 +56,9 @@ std::pair<String, StoragePtr> createTableFromAST(
|
||||
ast_create_query.attach = true;
|
||||
ast_create_query.setDatabase(database_name);
|
||||
|
||||
if (ast_create_query.select && ast_create_query.isView())
|
||||
ApplyWithSubqueryVisitor().visit(*ast_create_query.select);
|
||||
|
||||
if (ast_create_query.as_table_function)
|
||||
{
|
||||
const auto & factory = TableFunctionFactory::instance();
|
||||
|
@ -179,8 +179,12 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr
|
||||
|
||||
if (!task->was_executed)
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Entry {} was executed, but was not committed: code {}: {}",
|
||||
task->execution_status.code, task->execution_status.message);
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Entry {} was executed, but was not committed: code {}: {}",
|
||||
task->entry_name,
|
||||
task->execution_status.code,
|
||||
task->execution_status.message);
|
||||
}
|
||||
|
||||
try_node->setAlreadyRemoved();
|
||||
|
@ -50,7 +50,7 @@ namespace
|
||||
{
|
||||
if (!qualified_name.database.empty())
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Dictionary source of type {} specifies a schema but schema is not supported by {}-driver",
|
||||
"Dictionary source specifies a schema but schema is not supported by {}-driver",
|
||||
bridge_.getName());
|
||||
}
|
||||
|
||||
|
@ -392,8 +392,13 @@ void CachedReadBufferFromRemoteFS::predownload(FileSegmentPtr & file_segment)
|
||||
if (bytes_to_predownload)
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Failed to predownload remaining {} bytes. Current file segment: {}, current download offset: {}, expected: {}, eof: {}",
|
||||
file_segment->range().toString(), file_segment->getDownloadOffset(), file_offset_of_buffer_end, implementation_buffer->eof());
|
||||
"Failed to predownload remaining {} bytes. Current file segment: {}, current download offset: {}, expected: {}, "
|
||||
"eof: {}",
|
||||
bytes_to_predownload,
|
||||
file_segment->range().toString(),
|
||||
file_segment->getDownloadOffset(),
|
||||
file_offset_of_buffer_end,
|
||||
implementation_buffer->eof());
|
||||
|
||||
auto result = implementation_buffer->hasPendingData();
|
||||
|
||||
|
@ -44,7 +44,7 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S
|
||||
{
|
||||
return std::make_unique<ReadBufferFromS3>(
|
||||
client_ptr, bucket, fs::path(metadata.remote_fs_root_path) / path, max_single_read_retries,
|
||||
settings, /* use_external_buffer */true, read_until_position, /* restricted_seek */true);
|
||||
settings, /* use_external_buffer */true, /* offset */ 0, read_until_position, /* restricted_seek */true);
|
||||
};
|
||||
|
||||
if (with_cache)
|
||||
|
@ -85,9 +85,12 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String &
|
||||
else if (path.has_parent_path() && !fs::weakly_canonical(default_schema_directory_path / path).string().starts_with(fs::weakly_canonical(default_schema_directory_path).string()))
|
||||
{
|
||||
if (is_server)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: {} ({} not in {})",
|
||||
path.string());
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: {} ({} not in {})",
|
||||
default_schema_directory(),
|
||||
path.string(),
|
||||
default_schema_directory());
|
||||
path = default_schema_directory_path / path;
|
||||
schema_path = path.filename();
|
||||
schema_directory = path.parent_path() / "";
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeObject.h>
|
||||
#include <Common/JSONParsers/SimdJSONParser.h>
|
||||
#include <Common/JSONParsers/RapidJSONParser.h>
|
||||
#include <Common/JSONParsers/DummyJSONParser.h>
|
||||
@ -158,22 +159,37 @@ DataTypePtr getDataTypeFromJSONFieldImpl(const Element & field)
|
||||
{
|
||||
auto object = field.getObject();
|
||||
DataTypePtr value_type;
|
||||
bool is_object = false;
|
||||
for (const auto key_value_pair : object)
|
||||
{
|
||||
auto type = getDataTypeFromJSONFieldImpl(key_value_pair.second);
|
||||
if (!type)
|
||||
return nullptr;
|
||||
continue;
|
||||
|
||||
if (value_type && value_type->getName() != type->getName())
|
||||
return nullptr;
|
||||
if (isObject(type))
|
||||
{
|
||||
is_object = true;
|
||||
break;
|
||||
}
|
||||
|
||||
value_type = type;
|
||||
if (!value_type)
|
||||
{
|
||||
value_type = type;
|
||||
}
|
||||
else if (!value_type->equals(*type))
|
||||
{
|
||||
is_object = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!value_type)
|
||||
return nullptr;
|
||||
if (is_object)
|
||||
return std::make_shared<DataTypeObject>("json", false);
|
||||
|
||||
return std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), value_type);
|
||||
if (value_type)
|
||||
return std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), value_type);
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
throw Exception{ErrorCodes::INCORRECT_DATA, "Unexpected JSON type"};
|
||||
|
@ -7,6 +7,8 @@
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -17,6 +19,28 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
static std::optional<NamesAndTypesList> getOrderedColumnsList(
|
||||
const NamesAndTypesList & columns_list, const Names & columns_order_hint)
|
||||
{
|
||||
if (columns_list.size() != columns_order_hint.size())
|
||||
return {};
|
||||
|
||||
std::unordered_map<String, DataTypePtr> available_columns;
|
||||
for (const auto & [name, type] : columns_list)
|
||||
available_columns.emplace(name, type);
|
||||
|
||||
NamesAndTypesList res;
|
||||
for (const auto & name : columns_order_hint)
|
||||
{
|
||||
auto it = available_columns.find(name);
|
||||
if (it == available_columns.end())
|
||||
return {};
|
||||
|
||||
res.emplace_back(name, it->second);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
ColumnsDescription readSchemaFromFormat(
|
||||
const String & format_name,
|
||||
const std::optional<FormatSettings> & format_settings,
|
||||
@ -52,6 +76,22 @@ ColumnsDescription readSchemaFromFormat(
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}", format_name, e.message());
|
||||
}
|
||||
|
||||
/// If we have "INSERT SELECT" query then try to order
|
||||
/// columns as they are ordered in table schema for formats
|
||||
/// without strict column order (like JSON and TSKV).
|
||||
/// It will allow to execute simple data loading with query
|
||||
/// "INSERT INTO table SELECT * FROM ..."
|
||||
const auto & insertion_table = context->getInsertionTable();
|
||||
if (!schema_reader->hasStrictOrderOfColumns() && !insertion_table.empty())
|
||||
{
|
||||
auto storage = DatabaseCatalog::instance().getTable(insertion_table, context);
|
||||
auto metadata = storage->getInMemoryMetadataPtr();
|
||||
auto names_in_storage = metadata->getColumns().getNamesOfPhysical();
|
||||
auto ordered_list = getOrderedColumnsList(names_and_types, names_in_storage);
|
||||
if (ordered_list)
|
||||
names_and_types = *ordered_list;
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} file format doesn't support schema inference", format_name);
|
||||
|
@ -53,6 +53,7 @@
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -886,7 +887,7 @@ struct ConvertImplGenericToString
|
||||
const IColumn & col_from = *col_with_type_and_name.column;
|
||||
|
||||
size_t size = col_from.size();
|
||||
auto col_to = result_type->createColumn();
|
||||
auto col_to = removeNullable(result_type)->createColumn();
|
||||
|
||||
{
|
||||
ColumnStringHelpers::WriteHelper write_helper(
|
||||
@ -3140,52 +3141,138 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
WrapperType createTupleToObjectWrapper(const DataTypeTuple & from_tuple, bool has_nullable_subcolumns) const
|
||||
{
|
||||
if (!from_tuple.haveExplicitNames())
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"Cast to Object can be performed only from flatten Named Tuple. Got: {}", from_tuple.getName());
|
||||
|
||||
PathsInData paths;
|
||||
DataTypes from_types;
|
||||
|
||||
std::tie(paths, from_types) = flattenTuple(from_tuple.getPtr());
|
||||
auto to_types = from_types;
|
||||
|
||||
for (auto & type : to_types)
|
||||
{
|
||||
if (isTuple(type) || isNested(type))
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"Cast to Object can be performed only from flatten Named Tuple. Got: {}",
|
||||
from_tuple.getName());
|
||||
|
||||
type = recursiveRemoveLowCardinality(type);
|
||||
}
|
||||
|
||||
return [element_wrappers = getElementWrappers(from_types, to_types),
|
||||
has_nullable_subcolumns, from_types, to_types, paths]
|
||||
(ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t input_rows_count)
|
||||
{
|
||||
size_t tuple_size = to_types.size();
|
||||
auto flattened_column = flattenTuple(arguments.front().column);
|
||||
const auto & column_tuple = assert_cast<const ColumnTuple &>(*flattened_column);
|
||||
|
||||
if (tuple_size != column_tuple.getColumns().size())
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"Expected tuple with {} subcolumn, but got {} subcolumns",
|
||||
tuple_size, column_tuple.getColumns().size());
|
||||
|
||||
auto res = ColumnObject::create(has_nullable_subcolumns);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
{
|
||||
ColumnsWithTypeAndName element = {{column_tuple.getColumns()[i], from_types[i], "" }};
|
||||
auto converted_column = element_wrappers[i](element, to_types[i], nullable_source, input_rows_count);
|
||||
res->addSubcolumn(paths[i], converted_column->assumeMutable());
|
||||
}
|
||||
|
||||
return res;
|
||||
};
|
||||
}
|
||||
|
||||
WrapperType createMapToObjectWrapper(const DataTypeMap & from_map, bool has_nullable_subcolumns) const
|
||||
{
|
||||
auto key_value_types = from_map.getKeyValueTypes();
|
||||
|
||||
if (!isStringOrFixedString(key_value_types[0]))
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"Cast to Object from Map can be performed only from Map "
|
||||
"with String or FixedString key. Got: {}", from_map.getName());
|
||||
|
||||
const auto & value_type = key_value_types[1];
|
||||
auto to_value_type = value_type;
|
||||
|
||||
if (!has_nullable_subcolumns && value_type->isNullable())
|
||||
to_value_type = removeNullable(value_type);
|
||||
|
||||
if (has_nullable_subcolumns && !value_type->isNullable())
|
||||
to_value_type = makeNullable(value_type);
|
||||
|
||||
DataTypes to_key_value_types{std::make_shared<DataTypeString>(), std::move(to_value_type)};
|
||||
auto element_wrappers = getElementWrappers(key_value_types, to_key_value_types);
|
||||
|
||||
return [has_nullable_subcolumns, element_wrappers, key_value_types, to_key_value_types]
|
||||
(ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t) -> ColumnPtr
|
||||
{
|
||||
const auto & column_map = assert_cast<const ColumnMap &>(*arguments.front().column);
|
||||
const auto & offsets = column_map.getNestedColumn().getOffsets();
|
||||
auto key_value_columns = column_map.getNestedData().getColumnsCopy();
|
||||
|
||||
for (size_t i = 0; i < 2; ++i)
|
||||
{
|
||||
ColumnsWithTypeAndName element{{key_value_columns[i], key_value_types[i], ""}};
|
||||
key_value_columns[i] = element_wrappers[i](element, to_key_value_types[i], nullable_source, key_value_columns[i]->size());
|
||||
}
|
||||
|
||||
const auto & key_column_str = assert_cast<const ColumnString &>(*key_value_columns[0]);
|
||||
const auto & value_column = *key_value_columns[1];
|
||||
|
||||
using SubcolumnsMap = HashMap<StringRef, MutableColumnPtr, StringRefHash>;
|
||||
SubcolumnsMap subcolumns;
|
||||
|
||||
for (size_t row = 0; row < offsets.size(); ++row)
|
||||
{
|
||||
for (size_t i = offsets[static_cast<ssize_t>(row) - 1]; i < offsets[row]; ++i)
|
||||
{
|
||||
auto ref = key_column_str.getDataAt(i);
|
||||
|
||||
bool inserted;
|
||||
SubcolumnsMap::LookupResult it;
|
||||
subcolumns.emplace(ref, it, inserted);
|
||||
auto & subcolumn = it->getMapped();
|
||||
|
||||
if (inserted)
|
||||
subcolumn = value_column.cloneEmpty()->cloneResized(row);
|
||||
|
||||
/// Map can have duplicated keys. We insert only first one.
|
||||
if (subcolumn->size() == row)
|
||||
subcolumn->insertFrom(value_column, i);
|
||||
}
|
||||
|
||||
/// Insert default values for keys missed in current row.
|
||||
for (const auto & [_, subcolumn] : subcolumns)
|
||||
if (subcolumn->size() == row)
|
||||
subcolumn->insertDefault();
|
||||
}
|
||||
|
||||
auto column_object = ColumnObject::create(has_nullable_subcolumns);
|
||||
for (auto && [key, subcolumn] : subcolumns)
|
||||
{
|
||||
PathInData path(key.toView());
|
||||
column_object->addSubcolumn(path, std::move(subcolumn));
|
||||
}
|
||||
|
||||
return column_object;
|
||||
};
|
||||
}
|
||||
|
||||
WrapperType createObjectWrapper(const DataTypePtr & from_type, const DataTypeObject * to_type) const
|
||||
{
|
||||
if (const auto * from_tuple = checkAndGetDataType<DataTypeTuple>(from_type.get()))
|
||||
{
|
||||
if (!from_tuple->haveExplicitNames())
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"Cast to Object can be performed only from flatten Named Tuple. Got: {}", from_type->getName());
|
||||
|
||||
PathsInData paths;
|
||||
DataTypes from_types;
|
||||
|
||||
std::tie(paths, from_types) = flattenTuple(from_type);
|
||||
auto to_types = from_types;
|
||||
|
||||
for (auto & type : to_types)
|
||||
{
|
||||
if (isTuple(type) || isNested(type))
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"Cast to Object can be performed only from flatten Named Tuple. Got: {}", from_type->getName());
|
||||
|
||||
type = recursiveRemoveLowCardinality(type);
|
||||
}
|
||||
|
||||
return [element_wrappers = getElementWrappers(from_types, to_types),
|
||||
has_nullable_subcolumns = to_type->hasNullableSubcolumns(), from_types, to_types, paths]
|
||||
(ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t input_rows_count)
|
||||
{
|
||||
size_t tuple_size = to_types.size();
|
||||
auto flattened_column = flattenTuple(arguments.front().column);
|
||||
const auto & column_tuple = assert_cast<const ColumnTuple &>(*flattened_column);
|
||||
|
||||
if (tuple_size != column_tuple.getColumns().size())
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"Expected tuple with {} subcolumn, but got {} subcolumns",
|
||||
tuple_size, column_tuple.getColumns().size());
|
||||
|
||||
auto res = ColumnObject::create(has_nullable_subcolumns);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
{
|
||||
ColumnsWithTypeAndName element = {{column_tuple.getColumns()[i], from_types[i], "" }};
|
||||
auto converted_column = element_wrappers[i](element, to_types[i], nullable_source, input_rows_count);
|
||||
res->addSubcolumn(paths[i], converted_column->assumeMutable());
|
||||
}
|
||||
|
||||
return res;
|
||||
};
|
||||
return createTupleToObjectWrapper(*from_tuple, to_type->hasNullableSubcolumns());
|
||||
}
|
||||
else if (const auto * from_map = checkAndGetDataType<DataTypeMap>(from_type.get()))
|
||||
{
|
||||
return createMapToObjectWrapper(*from_map, to_type->hasNullableSubcolumns());
|
||||
}
|
||||
else if (checkAndGetDataType<DataTypeString>(from_type.get()))
|
||||
{
|
||||
@ -3199,7 +3286,7 @@ private:
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"Cast to Object can be performed only from flatten named tuple or string. Got: {}", from_type->getName());
|
||||
"Cast to Object can be performed only from flatten named Tuple, Map or String. Got: {}", from_type->getName());
|
||||
}
|
||||
|
||||
template <typename FieldType>
|
||||
|
@ -259,7 +259,7 @@ public:
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Function '{}' needs at least 2 arguments, at most 3 arguments; passed {}.",
|
||||
arguments.size());
|
||||
name, arguments.size());
|
||||
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.",
|
||||
|
@ -181,9 +181,12 @@ ColumnPtr IExecutableFunction::defaultImplementationForNulls(
|
||||
// Default implementation for nulls returns null result for null arguments,
|
||||
// so the result type must be nullable.
|
||||
if (!result_type->isNullable())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Function {} with Null argument and default implementation for Nulls "
|
||||
"is expected to return Nullable result, got {}", result_type->getName());
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Function {} with Null argument and default implementation for Nulls "
|
||||
"is expected to return Nullable result, got {}",
|
||||
getName(),
|
||||
result_type->getName());
|
||||
|
||||
return result_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
}
|
||||
|
@ -231,7 +231,7 @@ private:
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Function {} decimal scale should have native UInt type. Actual {}",
|
||||
scale_argument.type->getName());
|
||||
getName(), scale_argument.type->getName());
|
||||
}
|
||||
|
||||
scale = arguments[additional_argument_index].column->getUInt(0);
|
||||
|
@ -52,23 +52,21 @@ public:
|
||||
{
|
||||
if (arguments.size() < 2)
|
||||
throw Exception(
|
||||
"Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size())
|
||||
+ ", should be at least 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (arguments.size() > FormatImpl::argument_threshold)
|
||||
throw Exception(
|
||||
"Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size())
|
||||
+ ", should be at most " + std::to_string(FormatImpl::argument_threshold),
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be at least 2",
|
||||
getName(),
|
||||
arguments.size());
|
||||
|
||||
for (const auto arg_idx : collections::range(0, arguments.size()))
|
||||
{
|
||||
const auto * arg = arguments[arg_idx].get();
|
||||
if (!isStringOrFixedString(arg))
|
||||
throw Exception{"Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function "
|
||||
+ getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument {} of function {}",
|
||||
arg->getName(),
|
||||
arg_idx + 1,
|
||||
getName());
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
@ -125,7 +123,7 @@ private:
|
||||
std::vector<const ColumnString::Chars *> data(num_arguments);
|
||||
std::vector<const ColumnString::Offsets *> offsets(num_arguments);
|
||||
std::vector<size_t> fixed_string_sizes(num_arguments);
|
||||
std::vector<String> constant_strings(num_arguments);
|
||||
std::vector<std::optional<String>> constant_strings(num_arguments);
|
||||
bool has_column_string = false;
|
||||
bool has_column_fixed_string = false;
|
||||
for (size_t i = 0; i < num_arguments; ++i)
|
||||
|
@ -112,7 +112,7 @@ public:
|
||||
|| (res = executeType<DataTypeDateTime64>(arguments, result_type))))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of function {], must be Date or DateTime.",
|
||||
"Illegal column {} of function {}, must be Date or DateTime.",
|
||||
arguments[1].column->getName(),
|
||||
getName());
|
||||
|
||||
|
68
src/Functions/flattenTuple.cpp
Normal file
68
src/Functions/flattenTuple.cpp
Normal file
@ -0,0 +1,68 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/ObjectUtils.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class FunctionFlattenTuple : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "flattenTuple";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionFlattenTuple>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return true; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
const auto & type = arguments[0];
|
||||
const auto * type_tuple = checkAndGetDataType<DataTypeTuple>(type.get());
|
||||
if (!type_tuple || !type_tuple->haveExplicitNames())
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Argument for function '{}' must be Named Tuple. Got '{}'",
|
||||
getName(), type->getName());
|
||||
|
||||
auto [paths, types] = flattenTuple(type);
|
||||
Names names;
|
||||
names.reserve(paths.size());
|
||||
for (const auto & path : paths)
|
||||
names.push_back(path.getPath());
|
||||
|
||||
return std::make_shared<DataTypeTuple>(types, names);
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
auto column = arguments.at(0).column;
|
||||
if (!checkAndGetColumn<ColumnTuple>(column.get()))
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of first argument of function {}. Expected ColumnTuple",
|
||||
column->getName(), getName());
|
||||
|
||||
return flattenTuple(column);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
void registerFunctionFlattenTuple(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionFlattenTuple>();
|
||||
}
|
||||
|
||||
}
|
@ -45,25 +45,23 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.empty())
|
||||
if (arguments.size() < 2)
|
||||
throw Exception(
|
||||
"Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size())
|
||||
+ ", should be at least 1",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (arguments.size() > FormatImpl::argument_threshold)
|
||||
throw Exception(
|
||||
"Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size())
|
||||
+ ", should be at most " + std::to_string(FormatImpl::argument_threshold),
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be at least 2",
|
||||
getName(),
|
||||
arguments.size());
|
||||
|
||||
for (const auto arg_idx : collections::range(0, arguments.size()))
|
||||
{
|
||||
const auto * arg = arguments[arg_idx].get();
|
||||
if (!isStringOrFixedString(arg))
|
||||
throw Exception(
|
||||
"Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument {} of function {}",
|
||||
arg->getName(),
|
||||
arg_idx + 1,
|
||||
getName());
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
@ -84,7 +82,7 @@ public:
|
||||
std::vector<const ColumnString::Chars *> data(arguments.size() - 1);
|
||||
std::vector<const ColumnString::Offsets *> offsets(arguments.size() - 1);
|
||||
std::vector<size_t> fixed_string_sizes(arguments.size() - 1);
|
||||
std::vector<String> constant_strings(arguments.size() - 1);
|
||||
std::vector<std::optional<String>> constant_strings(arguments.size() - 1);
|
||||
|
||||
bool has_column_string = false;
|
||||
bool has_column_fixed_string = false;
|
||||
|
@ -4,8 +4,10 @@
|
||||
#include <base/types.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/format.h>
|
||||
#include <Common/memcpySmall.h>
|
||||
|
||||
|
||||
#include <algorithm>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
@ -15,15 +17,9 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
struct FormatImpl
|
||||
{
|
||||
static constexpr size_t small_argument_threshold = 1024;
|
||||
static constexpr size_t argument_threshold = std::numeric_limits<UInt32>::max();
|
||||
static constexpr size_t right_padding = 15;
|
||||
|
||||
template <typename... Args>
|
||||
@ -39,165 +35,10 @@ struct FormatImpl
|
||||
format<false, false>(std::forward<Args>(args)...);
|
||||
}
|
||||
|
||||
static void parseNumber(const String & description, UInt64 l, UInt64 r, UInt64 & res)
|
||||
{
|
||||
res = 0;
|
||||
for (UInt64 pos = l; pos < r; ++pos)
|
||||
{
|
||||
if (!isNumericASCII(description[pos]))
|
||||
throw Exception("Not a number in curly braces at position " + std::to_string(pos), ErrorCodes::BAD_ARGUMENTS);
|
||||
res = res * 10 + description[pos] - '0';
|
||||
if (res >= argument_threshold)
|
||||
throw Exception(
|
||||
"Too big number for arguments, must be at most " + std::to_string(argument_threshold), ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
}
|
||||
|
||||
static inline void init(
|
||||
const String & pattern,
|
||||
const std::vector<const ColumnString::Chars *> & data,
|
||||
size_t argument_number,
|
||||
const std::vector<String> & constant_strings,
|
||||
UInt64 * index_positions_ptr,
|
||||
std::vector<String> & substrings)
|
||||
{
|
||||
/// Is current position after open curly brace.
|
||||
bool is_open_curly = false;
|
||||
/// The position of last open token.
|
||||
size_t last_open = -1;
|
||||
|
||||
/// Is formatting in a plain {} token.
|
||||
std::optional<bool> is_plain_numbering;
|
||||
UInt64 index_if_plain = 0;
|
||||
|
||||
/// Left position of adding substrings, just to the closed brace position or the start of the string.
|
||||
/// Invariant --- the start of substring is in this position.
|
||||
size_t start_pos = 0;
|
||||
|
||||
/// A flag to decide whether we should glue the constant strings.
|
||||
bool glue_to_next = false;
|
||||
|
||||
/// Handling double braces (escaping).
|
||||
auto double_brace_removal = [](String & str)
|
||||
{
|
||||
size_t i = 0;
|
||||
bool should_delete = true;
|
||||
str.erase(
|
||||
std::remove_if(
|
||||
str.begin(),
|
||||
str.end(),
|
||||
[&i, &should_delete, &str](char)
|
||||
{
|
||||
bool is_double_brace = (str[i] == '{' && str[i + 1] == '{') || (str[i] == '}' && str[i + 1] == '}');
|
||||
++i;
|
||||
if (is_double_brace && should_delete)
|
||||
{
|
||||
should_delete = false;
|
||||
return true;
|
||||
}
|
||||
should_delete = true;
|
||||
return false;
|
||||
}),
|
||||
str.end());
|
||||
};
|
||||
|
||||
for (size_t i = 0; i < pattern.size(); ++i)
|
||||
{
|
||||
if (pattern[i] == '{')
|
||||
{
|
||||
/// Escaping handling
|
||||
/// It is safe to access because of null termination
|
||||
if (pattern[i + 1] == '{')
|
||||
{
|
||||
++i;
|
||||
continue;
|
||||
}
|
||||
|
||||
if (is_open_curly)
|
||||
throw Exception("Two open curly braces without close one at position " + std::to_string(i), ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
String to_add = String(pattern.data() + start_pos, i - start_pos);
|
||||
double_brace_removal(to_add);
|
||||
if (!glue_to_next)
|
||||
substrings.emplace_back(to_add);
|
||||
else
|
||||
substrings.back() += to_add;
|
||||
|
||||
glue_to_next = false;
|
||||
|
||||
is_open_curly = true;
|
||||
last_open = i + 1;
|
||||
}
|
||||
else if (pattern[i] == '}')
|
||||
{
|
||||
if (pattern[i + 1] == '}')
|
||||
{
|
||||
++i;
|
||||
continue;
|
||||
}
|
||||
|
||||
if (!is_open_curly)
|
||||
throw Exception("Closed curly brace without open one at position " + std::to_string(i), ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
is_open_curly = false;
|
||||
|
||||
if (last_open == i)
|
||||
{
|
||||
if (is_plain_numbering && !*is_plain_numbering)
|
||||
throw Exception(
|
||||
"Cannot switch from automatic field numbering to manual field specification", ErrorCodes::BAD_ARGUMENTS);
|
||||
is_plain_numbering = true;
|
||||
if (index_if_plain >= argument_number)
|
||||
throw Exception("Argument is too big for formatting", ErrorCodes::BAD_ARGUMENTS);
|
||||
*index_positions_ptr = index_if_plain++;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (is_plain_numbering && *is_plain_numbering)
|
||||
throw Exception(
|
||||
"Cannot switch from automatic field numbering to manual field specification", ErrorCodes::BAD_ARGUMENTS);
|
||||
is_plain_numbering = false;
|
||||
|
||||
UInt64 arg;
|
||||
parseNumber(pattern, last_open, i, arg);
|
||||
|
||||
if (arg >= argument_number)
|
||||
throw Exception(
|
||||
"Argument is too big for formatting. Note that indexing starts from zero", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
*index_positions_ptr = arg;
|
||||
}
|
||||
|
||||
/// Constant string.
|
||||
if (!data[*index_positions_ptr])
|
||||
{
|
||||
/// The next string should be glued to last `A {} C`.format('B') -> `A B C`.
|
||||
glue_to_next = true;
|
||||
substrings.back() += constant_strings[*index_positions_ptr];
|
||||
}
|
||||
else
|
||||
++index_positions_ptr; /// Otherwise we commit arg number and proceed.
|
||||
|
||||
start_pos = i + 1;
|
||||
}
|
||||
}
|
||||
|
||||
if (is_open_curly)
|
||||
throw Exception("Last open curly brace is not closed", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
String to_add = String(pattern.data() + start_pos, pattern.size() - start_pos);
|
||||
double_brace_removal(to_add);
|
||||
|
||||
if (!glue_to_next)
|
||||
substrings.emplace_back(to_add);
|
||||
else
|
||||
substrings.back() += to_add;
|
||||
}
|
||||
|
||||
/// data for ColumnString and ColumnFixed. Nullptr means no data, it is const string.
|
||||
/// offsets for ColumnString, nullptr is an indicator that there is a fixed string rather than ColumnString.
|
||||
/// fixed_string_N for savings N to fixed strings.
|
||||
/// constant_strings for constant strings. If data[i] is nullptr, than it is constant string.
|
||||
/// constant_strings for constant strings. If data[i] is nullptr, it is constant string.
|
||||
/// res_data is result_data, res_offsets is offset result.
|
||||
/// input_rows_count is the number of rows processed.
|
||||
/// Precondition: data.size() == offsets.size() == fixed_string_N.size() == constant_strings.size().
|
||||
@ -207,29 +48,22 @@ struct FormatImpl
|
||||
const std::vector<const ColumnString::Chars *> & data,
|
||||
const std::vector<const ColumnString::Offsets *> & offsets,
|
||||
[[maybe_unused]] /* Because sometimes !has_column_fixed_string */ const std::vector<size_t> & fixed_string_N,
|
||||
const std::vector<String> & constant_strings,
|
||||
const std::vector<std::optional<String>> & constant_strings,
|
||||
ColumnString::Chars & res_data,
|
||||
ColumnString::Offsets & res_offsets,
|
||||
size_t input_rows_count)
|
||||
{
|
||||
const size_t argument_number = offsets.size();
|
||||
|
||||
UInt64 small_index_positions_buffer[small_argument_threshold];
|
||||
/// The subsequent indexes of strings we should use. e.g `Hello world {1} {3} {1} {0}` this array will be filled with [1, 3, 1, 0, ... (garbage)] but without constant string indices.
|
||||
UInt64 * index_positions = small_index_positions_buffer;
|
||||
|
||||
std::unique_ptr<UInt64[]> big_index_positions_buffer;
|
||||
if (argument_number > small_argument_threshold)
|
||||
{
|
||||
big_index_positions_buffer.reset(new UInt64[argument_number]);
|
||||
index_positions = big_index_positions_buffer.get();
|
||||
}
|
||||
/// The subsequent indexes of strings we should use. e.g `Hello world {1} {3} {1} {0}` this
|
||||
/// array will be filled with [1, 3, 1, 0] but without constant string indices.
|
||||
Format::IndexPositions index_positions;
|
||||
|
||||
/// Vector of substrings of pattern that will be copied to the answer, not string view because of escaping and iterators invalidation.
|
||||
/// These are exactly what is between {} tokens, for `Hello {} world {}` we will have [`Hello `, ` world `, ``].
|
||||
std::vector<String> substrings;
|
||||
|
||||
init(pattern, data, argument_number, constant_strings, index_positions, substrings);
|
||||
Format::init(pattern, argument_number, constant_strings, index_positions, substrings);
|
||||
|
||||
UInt64 final_size = 0;
|
||||
|
||||
@ -271,7 +105,7 @@ struct FormatImpl
|
||||
for (size_t j = 1; j < substrings.size(); ++j)
|
||||
{
|
||||
UInt64 arg = index_positions[j - 1];
|
||||
auto offset_ptr = offsets[arg];
|
||||
const auto * offset_ptr = offsets[arg];
|
||||
UInt64 arg_offset = 0;
|
||||
UInt64 size = 0;
|
||||
|
||||
|
@ -81,6 +81,7 @@ void registerFunctionInitialQueryID(FunctionFactory & factory);
|
||||
void registerFunctionServerUUID(FunctionFactory &);
|
||||
void registerFunctionZooKeeperSessionUptime(FunctionFactory &);
|
||||
void registerFunctionGetOSKernelVersion(FunctionFactory &);
|
||||
void registerFunctionFlattenTuple(FunctionFactory &);
|
||||
|
||||
#if USE_ICU
|
||||
void registerFunctionConvertCharset(FunctionFactory &);
|
||||
@ -168,6 +169,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
||||
registerFunctionServerUUID(factory);
|
||||
registerFunctionZooKeeperSessionUptime(factory);
|
||||
registerFunctionGetOSKernelVersion(factory);
|
||||
registerFunctionFlattenTuple(factory);
|
||||
|
||||
#if USE_ICU
|
||||
registerFunctionConvertCharset(factory);
|
||||
|
@ -13,18 +13,11 @@ namespace ErrorCodes
|
||||
|
||||
}
|
||||
|
||||
ParallelReadBuffer::ParallelReadBuffer(
|
||||
std::unique_ptr<ReadBufferFactory> reader_factory_,
|
||||
ThreadPool * pool_,
|
||||
size_t max_working_readers_,
|
||||
WorkerSetup worker_setup_,
|
||||
WorkerCleanup worker_cleanup_)
|
||||
ParallelReadBuffer::ParallelReadBuffer(std::unique_ptr<ReadBufferFactory> reader_factory_, CallbackRunner schedule_, size_t max_working_readers_)
|
||||
: SeekableReadBufferWithSize(nullptr, 0)
|
||||
, pool(pool_)
|
||||
, max_working_readers(max_working_readers_)
|
||||
, schedule(std::move(schedule_))
|
||||
, reader_factory(std::move(reader_factory_))
|
||||
, worker_setup(std::move(worker_setup_))
|
||||
, worker_cleanup(std::move(worker_cleanup_))
|
||||
{
|
||||
std::unique_lock<std::mutex> lock{mutex};
|
||||
addReaders(lock);
|
||||
@ -40,30 +33,8 @@ bool ParallelReadBuffer::addReaderToPool(std::unique_lock<std::mutex> & /*buffer
|
||||
|
||||
auto worker = read_workers.emplace_back(std::make_shared<ReadWorker>(std::move(reader)));
|
||||
|
||||
pool->scheduleOrThrow(
|
||||
[&, this, worker = std::move(worker)]() mutable
|
||||
{
|
||||
ThreadStatus thread_status;
|
||||
schedule([this, worker = std::move(worker)]() mutable { readerThreadFunction(std::move(worker)); });
|
||||
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
++active_working_reader;
|
||||
}
|
||||
|
||||
SCOPE_EXIT({
|
||||
worker_cleanup(thread_status);
|
||||
|
||||
std::lock_guard lock{mutex};
|
||||
--active_working_reader;
|
||||
if (active_working_reader == 0)
|
||||
{
|
||||
readers_done.notify_all();
|
||||
}
|
||||
});
|
||||
worker_setup(thread_status);
|
||||
|
||||
readerThreadFunction(std::move(worker));
|
||||
});
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -232,12 +203,27 @@ bool ParallelReadBuffer::nextImpl()
|
||||
|
||||
void ParallelReadBuffer::readerThreadFunction(ReadWorkerPtr read_worker)
|
||||
{
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
++active_working_reader;
|
||||
}
|
||||
|
||||
SCOPE_EXIT({
|
||||
std::lock_guard lock{mutex};
|
||||
--active_working_reader;
|
||||
if (active_working_reader == 0)
|
||||
{
|
||||
readers_done.notify_all();
|
||||
}
|
||||
});
|
||||
|
||||
try
|
||||
{
|
||||
while (!emergency_stop && !read_worker->cancel)
|
||||
{
|
||||
if (!read_worker->reader->next())
|
||||
throw Exception("Failed to read all the data from the reader", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR, "Failed to read all the data from the reader, missing {} bytes", read_worker->bytes_left);
|
||||
|
||||
if (emergency_stop || read_worker->cancel)
|
||||
break;
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <IO/BufferWithOwnMemory.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/SeekableReadBuffer.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Common/ArenaWithFreeLists.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
|
||||
@ -76,14 +77,7 @@ public:
|
||||
virtual std::optional<size_t> getTotalSize() = 0;
|
||||
};
|
||||
|
||||
using WorkerSetup = std::function<void(ThreadStatus &)>;
|
||||
using WorkerCleanup = std::function<void(ThreadStatus &)>;
|
||||
explicit ParallelReadBuffer(
|
||||
std::unique_ptr<ReadBufferFactory> reader_factory_,
|
||||
ThreadPool * pool,
|
||||
size_t max_working_readers,
|
||||
WorkerSetup worker_setup = {},
|
||||
WorkerCleanup worker_cleanup = {});
|
||||
explicit ParallelReadBuffer(std::unique_ptr<ReadBufferFactory> reader_factory_, CallbackRunner schedule_, size_t max_working_readers);
|
||||
|
||||
~ParallelReadBuffer() override { finishAndWait(); }
|
||||
|
||||
@ -140,16 +134,14 @@ private:
|
||||
|
||||
Segment current_segment;
|
||||
|
||||
ThreadPool * pool;
|
||||
size_t max_working_readers;
|
||||
size_t active_working_reader{0};
|
||||
// Triggered when all reader workers are done
|
||||
std::condition_variable readers_done;
|
||||
|
||||
std::unique_ptr<ReadBufferFactory> reader_factory;
|
||||
CallbackRunner schedule;
|
||||
|
||||
WorkerSetup worker_setup;
|
||||
WorkerCleanup worker_cleanup;
|
||||
std::unique_ptr<ReadBufferFactory> reader_factory;
|
||||
|
||||
/**
|
||||
* FIFO queue of readers.
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Common/config.h>
|
||||
#include "IO/S3Common.h"
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
@ -42,6 +43,7 @@ ReadBufferFromS3::ReadBufferFromS3(
|
||||
UInt64 max_single_read_retries_,
|
||||
const ReadSettings & settings_,
|
||||
bool use_external_buffer_,
|
||||
size_t offset_,
|
||||
size_t read_until_position_,
|
||||
bool restricted_seek_)
|
||||
: SeekableReadBufferWithSize(nullptr, 0)
|
||||
@ -49,9 +51,10 @@ ReadBufferFromS3::ReadBufferFromS3(
|
||||
, bucket(bucket_)
|
||||
, key(key_)
|
||||
, max_single_read_retries(max_single_read_retries_)
|
||||
, offset(offset_)
|
||||
, read_until_position(read_until_position_)
|
||||
, read_settings(settings_)
|
||||
, use_external_buffer(use_external_buffer_)
|
||||
, read_until_position(read_until_position_)
|
||||
, restricted_seek(restricted_seek_)
|
||||
{
|
||||
}
|
||||
@ -210,13 +213,14 @@ std::optional<size_t> ReadBufferFromS3::getTotalSize()
|
||||
if (file_size)
|
||||
return file_size;
|
||||
|
||||
Aws::S3::Model::HeadObjectRequest request;
|
||||
request.SetBucket(bucket);
|
||||
request.SetKey(key);
|
||||
auto object_size = S3::getObjectSize(client_ptr, bucket, key, false);
|
||||
|
||||
auto outcome = client_ptr->HeadObject(request);
|
||||
auto head_result = outcome.GetResultWithOwnership();
|
||||
file_size = head_result.GetContentLength();
|
||||
if (!object_size)
|
||||
{
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
file_size = object_size;
|
||||
return file_size;
|
||||
}
|
||||
|
||||
@ -234,6 +238,11 @@ void ReadBufferFromS3::setReadUntilPosition(size_t position)
|
||||
}
|
||||
}
|
||||
|
||||
SeekableReadBuffer::Range ReadBufferFromS3::getRemainingReadRange() const
|
||||
{
|
||||
return Range{.left = static_cast<size_t>(offset), .right = read_until_position ? std::optional{read_until_position - 1} : std::nullopt};
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBuffer> ReadBufferFromS3::initialize()
|
||||
{
|
||||
Aws::S3::Model::GetObjectRequest req;
|
||||
@ -272,6 +281,36 @@ std::unique_ptr<ReadBuffer> ReadBufferFromS3::initialize()
|
||||
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
|
||||
}
|
||||
|
||||
SeekableReadBufferPtr ReadBufferS3Factory::getReader()
|
||||
{
|
||||
const auto next_range = range_generator.nextRange();
|
||||
if (!next_range)
|
||||
{
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
auto reader = std::make_shared<ReadBufferFromS3>(
|
||||
client_ptr,
|
||||
bucket,
|
||||
key,
|
||||
s3_max_single_read_retries,
|
||||
read_settings,
|
||||
false /*use_external_buffer*/,
|
||||
next_range->first,
|
||||
next_range->second);
|
||||
return reader;
|
||||
}
|
||||
|
||||
off_t ReadBufferS3Factory::seek(off_t off, [[maybe_unused]] int whence)
|
||||
{
|
||||
range_generator = RangeGenerator{object_size, range_step, static_cast<size_t>(off)};
|
||||
return off;
|
||||
}
|
||||
|
||||
std::optional<size_t> ReadBufferS3Factory::getTotalSize()
|
||||
{
|
||||
return object_size;
|
||||
}
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/RangeGenerator.h>
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AWS_S3
|
||||
@ -7,6 +8,7 @@
|
||||
#include <memory>
|
||||
|
||||
#include <IO/HTTPCommon.h>
|
||||
#include <IO/ParallelReadBuffer.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <IO/SeekableReadBuffer.h>
|
||||
@ -30,7 +32,9 @@ private:
|
||||
String bucket;
|
||||
String key;
|
||||
UInt64 max_single_read_retries;
|
||||
|
||||
off_t offset = 0;
|
||||
off_t read_until_position = 0;
|
||||
|
||||
Aws::S3::Model::GetObjectResult read_result;
|
||||
std::unique_ptr<ReadBuffer> impl;
|
||||
@ -45,6 +49,7 @@ public:
|
||||
UInt64 max_single_read_retries_,
|
||||
const ReadSettings & settings_,
|
||||
bool use_external_buffer = false,
|
||||
size_t offset_ = 0,
|
||||
size_t read_until_position_ = 0,
|
||||
bool restricted_seek_ = false);
|
||||
|
||||
@ -58,7 +63,7 @@ public:
|
||||
|
||||
void setReadUntilPosition(size_t position) override;
|
||||
|
||||
Range getRemainingReadRange() const override { return Range{ .left = static_cast<size_t>(offset), .right = read_until_position }; }
|
||||
Range getRemainingReadRange() const override;
|
||||
|
||||
size_t getFileOffsetOfBufferEnd() const override { return offset; }
|
||||
|
||||
@ -69,13 +74,55 @@ private:
|
||||
|
||||
bool use_external_buffer;
|
||||
|
||||
off_t read_until_position = 0;
|
||||
|
||||
/// There is different seek policy for disk seek and for non-disk seek
|
||||
/// (non-disk seek is applied for seekable input formats: orc, arrow, parquet).
|
||||
bool restricted_seek;
|
||||
};
|
||||
|
||||
/// Creates separate ReadBufferFromS3 for sequence of ranges of particular object
|
||||
class ReadBufferS3Factory : public ParallelReadBuffer::ReadBufferFactory
|
||||
{
|
||||
public:
|
||||
explicit ReadBufferS3Factory(
|
||||
std::shared_ptr<Aws::S3::S3Client> client_ptr_,
|
||||
const String & bucket_,
|
||||
const String & key_,
|
||||
size_t range_step_,
|
||||
size_t object_size_,
|
||||
UInt64 s3_max_single_read_retries_,
|
||||
const ReadSettings & read_settings_)
|
||||
: client_ptr(client_ptr_)
|
||||
, bucket(bucket_)
|
||||
, key(key_)
|
||||
, read_settings(read_settings_)
|
||||
, range_generator(object_size_, range_step_)
|
||||
, range_step(range_step_)
|
||||
, object_size(object_size_)
|
||||
, s3_max_single_read_retries(s3_max_single_read_retries_)
|
||||
{
|
||||
assert(range_step > 0);
|
||||
assert(range_step < object_size);
|
||||
}
|
||||
|
||||
SeekableReadBufferPtr getReader() override;
|
||||
|
||||
off_t seek(off_t off, [[maybe_unused]] int whence) override;
|
||||
|
||||
std::optional<size_t> getTotalSize() override;
|
||||
|
||||
private:
|
||||
std::shared_ptr<Aws::S3::S3Client> client_ptr;
|
||||
const String bucket;
|
||||
const String key;
|
||||
ReadSettings read_settings;
|
||||
|
||||
RangeGenerator range_generator;
|
||||
size_t range_step;
|
||||
size_t object_size;
|
||||
|
||||
UInt64 s3_max_single_read_retries;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <functional>
|
||||
#include <Common/RangeGenerator.h>
|
||||
#include <IO/ConnectionTimeouts.h>
|
||||
#include <IO/HTTPCommon.h>
|
||||
#include <IO/ParallelReadBuffer.h>
|
||||
@ -635,43 +636,6 @@ public:
|
||||
void buildNewSession(const Poco::URI & uri) override { session = makeHTTPSession(uri, timeouts); }
|
||||
};
|
||||
|
||||
class RangeGenerator
|
||||
{
|
||||
public:
|
||||
explicit RangeGenerator(size_t total_size_, size_t range_step_, size_t range_start = 0)
|
||||
: from(range_start), range_step(range_step_), total_size(total_size_)
|
||||
{
|
||||
}
|
||||
|
||||
size_t totalRanges() const { return static_cast<size_t>(round(static_cast<float>(total_size - from) / range_step)); }
|
||||
|
||||
using Range = std::pair<size_t, size_t>;
|
||||
|
||||
// return upper exclusive range of values, i.e. [from_range, to_range>
|
||||
std::optional<Range> nextRange()
|
||||
{
|
||||
if (from >= total_size)
|
||||
{
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
auto to = from + range_step;
|
||||
if (to >= total_size)
|
||||
{
|
||||
to = total_size;
|
||||
}
|
||||
|
||||
Range range{from, to};
|
||||
from = to;
|
||||
return range;
|
||||
}
|
||||
|
||||
private:
|
||||
size_t from;
|
||||
size_t range_step;
|
||||
size_t total_size;
|
||||
};
|
||||
|
||||
class ReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<UpdatableSession>>
|
||||
{
|
||||
using Parent = detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<UpdatableSession>>;
|
||||
|
@ -24,6 +24,7 @@
|
||||
# include <aws/core/utils/UUID.h>
|
||||
# include <aws/core/http/HttpClientFactory.h>
|
||||
# include <aws/s3/S3Client.h>
|
||||
# include <aws/s3/model/HeadObjectRequest.h> // Y_IGNORE
|
||||
|
||||
# include <IO/S3/PocoHTTPClientFactory.h>
|
||||
# include <IO/S3/PocoHTTPClient.h>
|
||||
@ -682,6 +683,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int S3_ERROR;
|
||||
}
|
||||
|
||||
namespace S3
|
||||
@ -839,6 +841,26 @@ namespace S3
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bucket name length is out of bounds in virtual hosted style S3 URI: {}{}",
|
||||
quoteString(bucket), !uri.empty() ? " (" + uri.toString() + ")" : "");
|
||||
}
|
||||
|
||||
size_t getObjectSize(std::shared_ptr<Aws::S3::S3Client> client_ptr, const String & bucket, const String & key, bool throw_on_error)
|
||||
{
|
||||
Aws::S3::Model::HeadObjectRequest req;
|
||||
req.SetBucket(bucket);
|
||||
req.SetKey(key);
|
||||
|
||||
Aws::S3::Model::HeadObjectOutcome outcome = client_ptr->HeadObject(req);
|
||||
|
||||
if (outcome.IsSuccess())
|
||||
{
|
||||
auto read_result = outcome.GetResultWithOwnership();
|
||||
return static_cast<size_t>(read_result.GetContentLength());
|
||||
}
|
||||
else if (throw_on_error)
|
||||
{
|
||||
throw DB::Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -75,6 +75,8 @@ struct URI
|
||||
static void validateBucket(const String & bucket, const Poco::URI & uri);
|
||||
};
|
||||
|
||||
size_t getObjectSize(std::shared_ptr<Aws::S3::S3Client> client_ptr, const String & bucket, const String & key, bool throw_on_error = true);
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -1,4 +1,6 @@
|
||||
#include <algorithm>
|
||||
#include <future>
|
||||
#include <numeric>
|
||||
#include <Poco/Util/Application.h>
|
||||
|
||||
#include <base/sort.h>
|
||||
@ -15,6 +17,7 @@
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
#include <Interpreters/Aggregator.h>
|
||||
#include <Common/LRUCache.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
@ -27,12 +30,236 @@
|
||||
#include <Interpreters/JIT/CompiledExpressionCache.h>
|
||||
#include <Core/ProtocolDefines.h>
|
||||
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event ExternalAggregationWritePart;
|
||||
extern const Event ExternalAggregationCompressedBytes;
|
||||
extern const Event ExternalAggregationUncompressedBytes;
|
||||
extern const Event ExternalAggregationWritePart;
|
||||
extern const Event ExternalAggregationCompressedBytes;
|
||||
extern const Event ExternalAggregationUncompressedBytes;
|
||||
extern const Event AggregationPreallocatedElementsInHashTables;
|
||||
extern const Event AggregationHashTablesInitializedAsTwoLevel;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
/** Collects observed HashMap-s sizes to avoid redundant intermediate resizes.
|
||||
*/
|
||||
class HashTablesStatistics
|
||||
{
|
||||
public:
|
||||
struct Entry
|
||||
{
|
||||
size_t sum_of_sizes; // used to determine if it's better to convert aggregation to two-level from the beginning
|
||||
size_t median_size; // roughly the size we're going to preallocate on each thread
|
||||
};
|
||||
|
||||
using Cache = DB::LRUCache<UInt64, Entry>;
|
||||
using CachePtr = std::shared_ptr<Cache>;
|
||||
using Params = DB::Aggregator::Params::StatsCollectingParams;
|
||||
|
||||
/// Collection and use of the statistics should be enabled.
|
||||
std::optional<Entry> getSizeHint(const Params & params)
|
||||
{
|
||||
if (!params.isCollectionAndUseEnabled())
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled.");
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
const auto cache = getHashTableStatsCache(params, lock);
|
||||
if (const auto hint = cache->get(params.key))
|
||||
{
|
||||
LOG_DEBUG(
|
||||
&Poco::Logger::get("Aggregator"),
|
||||
"An entry for key={} found in cache: sum_of_sizes={}, median_size={}",
|
||||
params.key,
|
||||
hint->sum_of_sizes,
|
||||
hint->median_size);
|
||||
return *hint;
|
||||
}
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
/// Collection and use of the statistics should be enabled.
|
||||
void update(size_t sum_of_sizes, size_t median_size, const Params & params)
|
||||
{
|
||||
if (!params.isCollectionAndUseEnabled())
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled.");
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
const auto cache = getHashTableStatsCache(params, lock);
|
||||
const auto hint = cache->get(params.key);
|
||||
// We'll maintain the maximum among all the observed values until the next prediction turns out to be too wrong.
|
||||
if (!hint || sum_of_sizes < hint->sum_of_sizes / 2 || hint->sum_of_sizes < sum_of_sizes || median_size < hint->median_size / 2
|
||||
|| hint->median_size < median_size)
|
||||
{
|
||||
LOG_DEBUG(
|
||||
&Poco::Logger::get("Aggregator"),
|
||||
"Statistics updated for key={}: new sum_of_sizes={}, median_size={}",
|
||||
params.key,
|
||||
sum_of_sizes,
|
||||
median_size);
|
||||
cache->set(params.key, std::make_shared<Entry>(Entry{.sum_of_sizes = sum_of_sizes, .median_size = median_size}));
|
||||
}
|
||||
}
|
||||
|
||||
std::optional<DB::HashTablesCacheStatistics> getCacheStats() const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
if (hash_table_stats)
|
||||
{
|
||||
size_t hits = 0, misses = 0;
|
||||
hash_table_stats->getStats(hits, misses);
|
||||
return DB::HashTablesCacheStatistics{.entries = hash_table_stats->count(), .hits = hits, .misses = misses};
|
||||
}
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
static size_t calculateCacheKey(const DB::ASTPtr & select_query)
|
||||
{
|
||||
if (!select_query)
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Query ptr cannot be null");
|
||||
|
||||
const auto & select = select_query->as<DB::ASTSelectQuery &>();
|
||||
|
||||
// It may happen in some corner cases like `select 1 as num group by num`.
|
||||
if (!select.tables())
|
||||
return 0;
|
||||
|
||||
SipHash hash;
|
||||
hash.update(select.tables()->getTreeHash());
|
||||
if (const auto where = select.where())
|
||||
hash.update(where->getTreeHash());
|
||||
if (const auto group_by = select.groupBy())
|
||||
hash.update(group_by->getTreeHash());
|
||||
return hash.get64();
|
||||
}
|
||||
|
||||
private:
|
||||
CachePtr getHashTableStatsCache(const Params & params, const std::lock_guard<std::mutex> &)
|
||||
{
|
||||
if (!hash_table_stats || hash_table_stats->maxSize() != params.max_entries_for_hash_table_stats)
|
||||
hash_table_stats = std::make_shared<Cache>(params.max_entries_for_hash_table_stats);
|
||||
return hash_table_stats;
|
||||
}
|
||||
|
||||
mutable std::mutex mutex;
|
||||
CachePtr hash_table_stats;
|
||||
};
|
||||
|
||||
HashTablesStatistics & getHashTablesStatistics()
|
||||
{
|
||||
static HashTablesStatistics hash_tables_stats;
|
||||
return hash_tables_stats;
|
||||
}
|
||||
|
||||
bool worthConvertToTwoLevel(
|
||||
size_t group_by_two_level_threshold, size_t result_size, size_t group_by_two_level_threshold_bytes, auto result_size_bytes)
|
||||
{
|
||||
// params.group_by_two_level_threshold will be equal to 0 if we have only one thread to execute aggregation (refer to AggregatingStep::transformPipeline).
|
||||
return (group_by_two_level_threshold && result_size >= group_by_two_level_threshold)
|
||||
|| (group_by_two_level_threshold_bytes && result_size_bytes >= static_cast<Int64>(group_by_two_level_threshold_bytes));
|
||||
}
|
||||
|
||||
DB::AggregatedDataVariants::Type convertToTwoLevelTypeIfPossible(DB::AggregatedDataVariants::Type type)
|
||||
{
|
||||
using Type = DB::AggregatedDataVariants::Type;
|
||||
switch (type)
|
||||
{
|
||||
#define M(NAME) \
|
||||
case Type::NAME: \
|
||||
return Type::NAME##_two_level;
|
||||
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL(M)
|
||||
#undef M
|
||||
default:
|
||||
return type;
|
||||
}
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
void initDataVariantsWithSizeHint(
|
||||
DB::AggregatedDataVariants & result, DB::AggregatedDataVariants::Type method_chosen, const DB::Aggregator::Params & params)
|
||||
{
|
||||
const auto & stats_collecting_params = params.stats_collecting_params;
|
||||
if (stats_collecting_params.isCollectionAndUseEnabled())
|
||||
{
|
||||
if (auto hint = getHashTablesStatistics().getSizeHint(stats_collecting_params))
|
||||
{
|
||||
const auto max_threads = params.group_by_two_level_threshold != 0 ? std::max(params.max_threads, 1ul) : 1;
|
||||
const auto lower_limit = hint->sum_of_sizes / max_threads;
|
||||
const auto upper_limit = stats_collecting_params.max_size_to_preallocate_for_aggregation / max_threads;
|
||||
const auto adjusted = std::min(std::max(lower_limit, hint->median_size), upper_limit);
|
||||
if (worthConvertToTwoLevel(
|
||||
params.group_by_two_level_threshold,
|
||||
hint->sum_of_sizes,
|
||||
/*group_by_two_level_threshold_bytes*/ 0,
|
||||
/*result_size_bytes*/ 0))
|
||||
method_chosen = convertToTwoLevelTypeIfPossible(method_chosen);
|
||||
result.init(method_chosen, adjusted);
|
||||
ProfileEvents::increment(ProfileEvents::AggregationHashTablesInitializedAsTwoLevel, result.isTwoLevel());
|
||||
return;
|
||||
}
|
||||
}
|
||||
result.init(method_chosen);
|
||||
}
|
||||
|
||||
/// Collection and use of the statistics should be enabled.
|
||||
void updateStatistics(const DB::ManyAggregatedDataVariants & data_variants, const DB::Aggregator::Params::StatsCollectingParams & params)
|
||||
{
|
||||
if (!params.isCollectionAndUseEnabled())
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled.");
|
||||
|
||||
std::vector<size_t> sizes(data_variants.size());
|
||||
for (size_t i = 0; i < data_variants.size(); ++i)
|
||||
sizes[i] = data_variants[i]->size();
|
||||
const auto median_size = sizes.begin() + sizes.size() / 2; // not precisely though...
|
||||
std::nth_element(sizes.begin(), median_size, sizes.end());
|
||||
const auto sum_of_sizes = std::accumulate(sizes.begin(), sizes.end(), 0ull);
|
||||
getHashTablesStatistics().update(sum_of_sizes, *median_size, params);
|
||||
}
|
||||
|
||||
// The std::is_constructible trait isn't suitable here because some classes have template constructors with semantics different from providing size hints.
|
||||
// Also string hash table variants are not supported due to the fact that both local perf tests and tests in CI showed slowdowns for them.
|
||||
template <typename...>
|
||||
struct HasConstructorOfNumberOfElements : std::false_type
|
||||
{
|
||||
};
|
||||
|
||||
template <typename... Ts>
|
||||
struct HasConstructorOfNumberOfElements<HashMapTable<Ts...>> : std::true_type
|
||||
{
|
||||
};
|
||||
|
||||
template <typename Key, typename Cell, typename Hash, typename Grower, typename Allocator, template <typename...> typename ImplTable>
|
||||
struct HasConstructorOfNumberOfElements<TwoLevelHashMapTable<Key, Cell, Hash, Grower, Allocator, ImplTable>> : std::true_type
|
||||
{
|
||||
};
|
||||
|
||||
template <typename... Ts>
|
||||
struct HasConstructorOfNumberOfElements<HashTable<Ts...>> : std::true_type
|
||||
{
|
||||
};
|
||||
|
||||
template <typename... Ts>
|
||||
struct HasConstructorOfNumberOfElements<TwoLevelHashTable<Ts...>> : std::true_type
|
||||
{
|
||||
};
|
||||
|
||||
template <template <typename> typename Method, typename Base>
|
||||
struct HasConstructorOfNumberOfElements<Method<Base>> : HasConstructorOfNumberOfElements<Base>
|
||||
{
|
||||
};
|
||||
|
||||
template <typename Method>
|
||||
auto constructWithReserveIfPossible(size_t size_hint)
|
||||
{
|
||||
if constexpr (HasConstructorOfNumberOfElements<typename Method::Data>::value)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::AggregationPreallocatedElementsInHashTables, size_hint);
|
||||
return std::make_unique<Method>(size_hint);
|
||||
}
|
||||
else
|
||||
return std::make_unique<Method>();
|
||||
}
|
||||
}
|
||||
|
||||
namespace DB
|
||||
@ -64,6 +291,10 @@ AggregatedDataVariants::~AggregatedDataVariants()
|
||||
}
|
||||
}
|
||||
|
||||
std::optional<HashTablesCacheStatistics> getHashTablesCacheStatistics()
|
||||
{
|
||||
return getHashTablesStatistics().getCacheStats();
|
||||
}
|
||||
|
||||
void AggregatedDataVariants::convertToTwoLevel()
|
||||
{
|
||||
@ -88,6 +319,47 @@ void AggregatedDataVariants::convertToTwoLevel()
|
||||
}
|
||||
}
|
||||
|
||||
void AggregatedDataVariants::init(Type type_, std::optional<size_t> size_hint)
|
||||
{
|
||||
switch (type_)
|
||||
{
|
||||
case Type::EMPTY:
|
||||
break;
|
||||
case Type::without_key:
|
||||
break;
|
||||
|
||||
#define M(NAME, IS_TWO_LEVEL) \
|
||||
case Type::NAME: \
|
||||
if (size_hint) \
|
||||
(NAME) = constructWithReserveIfPossible<decltype(NAME)::element_type>(*size_hint); \
|
||||
else \
|
||||
(NAME) = std::make_unique<decltype(NAME)::element_type>(); \
|
||||
break;
|
||||
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
||||
#undef M
|
||||
}
|
||||
|
||||
type = type_;
|
||||
}
|
||||
|
||||
Aggregator::Params::StatsCollectingParams::StatsCollectingParams() = default;
|
||||
|
||||
Aggregator::Params::StatsCollectingParams::StatsCollectingParams(
|
||||
const ASTPtr & select_query_,
|
||||
bool collect_hash_table_stats_during_aggregation_,
|
||||
size_t max_entries_for_hash_table_stats_,
|
||||
size_t max_size_to_preallocate_for_aggregation_)
|
||||
: key(collect_hash_table_stats_during_aggregation_ ? HashTablesStatistics::calculateCacheKey(select_query_) : 0)
|
||||
, max_entries_for_hash_table_stats(max_entries_for_hash_table_stats_)
|
||||
, max_size_to_preallocate_for_aggregation(max_size_to_preallocate_for_aggregation_)
|
||||
{
|
||||
}
|
||||
|
||||
bool Aggregator::Params::StatsCollectingParams::isCollectionAndUseEnabled() const
|
||||
{
|
||||
return key != 0;
|
||||
}
|
||||
|
||||
Block Aggregator::getHeader(bool final) const
|
||||
{
|
||||
return params.getHeader(final);
|
||||
@ -237,8 +509,7 @@ public:
|
||||
|
||||
#endif
|
||||
|
||||
Aggregator::Aggregator(const Params & params_)
|
||||
: params(params_)
|
||||
Aggregator::Aggregator(const Params & params_) : params(params_)
|
||||
{
|
||||
/// Use query-level memory tracker
|
||||
if (auto * memory_tracker_child = CurrentThread::getMemoryTracker())
|
||||
@ -292,7 +563,6 @@ Aggregator::Aggregator(const Params & params_)
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
compileAggregateFunctionsIfNeeded();
|
||||
#endif
|
||||
|
||||
}
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
@ -958,7 +1228,7 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData
|
||||
/// How to perform the aggregation?
|
||||
if (result.empty())
|
||||
{
|
||||
result.init(method_chosen);
|
||||
initDataVariantsWithSizeHint(result, method_chosen, params);
|
||||
result.keys_size = params.keys_size;
|
||||
result.key_sizes = key_sizes;
|
||||
LOG_TRACE(log, "Aggregation method: {}", result.getMethodName());
|
||||
@ -1038,9 +1308,8 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData
|
||||
/// Here all the results in the sum are taken into account, from different threads.
|
||||
auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation;
|
||||
|
||||
bool worth_convert_to_two_level
|
||||
= (params.group_by_two_level_threshold && result_size >= params.group_by_two_level_threshold)
|
||||
|| (params.group_by_two_level_threshold_bytes && result_size_bytes >= static_cast<Int64>(params.group_by_two_level_threshold_bytes));
|
||||
bool worth_convert_to_two_level = worthConvertToTwoLevel(
|
||||
params.group_by_two_level_threshold, result_size, params.group_by_two_level_threshold_bytes, result_size_bytes);
|
||||
|
||||
/** Converting to a two-level data structure.
|
||||
* It allows you to make, in the subsequent, an effective merge - either economical from memory or parallel.
|
||||
@ -1327,10 +1596,7 @@ void Aggregator::convertToBlockImpl(
|
||||
|
||||
|
||||
template <typename Mapped>
|
||||
inline void Aggregator::insertAggregatesIntoColumns(
|
||||
Mapped & mapped,
|
||||
MutableColumns & final_aggregate_columns,
|
||||
Arena * arena) const
|
||||
inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColumns & final_aggregate_columns, Arena * arena) const
|
||||
{
|
||||
/** Final values of aggregate functions are inserted to columns.
|
||||
* Then states of aggregate functions, that are not longer needed, are destroyed.
|
||||
@ -2179,6 +2445,9 @@ ManyAggregatedDataVariants Aggregator::prepareVariantsToMerge(ManyAggregatedData
|
||||
|
||||
LOG_TRACE(log, "Merging aggregated data");
|
||||
|
||||
if (params.stats_collecting_params.isCollectionAndUseEnabled())
|
||||
updateStatistics(data_variants, params.stats_collecting_params);
|
||||
|
||||
ManyAggregatedDataVariants non_empty_data;
|
||||
non_empty_data.reserve(data_variants.size());
|
||||
for (auto & data : data_variants)
|
||||
@ -2388,9 +2657,8 @@ bool Aggregator::mergeOnBlock(Block block, AggregatedDataVariants & result, bool
|
||||
/// Here all the results in the sum are taken into account, from different threads.
|
||||
auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation;
|
||||
|
||||
bool worth_convert_to_two_level
|
||||
= (params.group_by_two_level_threshold && result_size >= params.group_by_two_level_threshold)
|
||||
|| (params.group_by_two_level_threshold_bytes && result_size_bytes >= static_cast<Int64>(params.group_by_two_level_threshold_bytes));
|
||||
bool worth_convert_to_two_level = worthConvertToTwoLevel(
|
||||
params.group_by_two_level_threshold, result_size, params.group_by_two_level_threshold_bytes, result_size_bytes);
|
||||
|
||||
/** Converting to a two-level data structure.
|
||||
* It allows you to make, in the subsequent, an effective merge - either economical from memory or parallel.
|
||||
|
@ -34,6 +34,7 @@
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -129,6 +130,7 @@ private:
|
||||
template <typename Base>
|
||||
struct AggregationDataWithNullKeyTwoLevel : public Base
|
||||
{
|
||||
using Base::Base;
|
||||
using Base::impls;
|
||||
|
||||
AggregationDataWithNullKeyTwoLevel() = default;
|
||||
@ -183,6 +185,8 @@ struct AggregationMethodOneNumber
|
||||
|
||||
AggregationMethodOneNumber() = default;
|
||||
|
||||
explicit AggregationMethodOneNumber(size_t size_hint) : data(size_hint) { }
|
||||
|
||||
template <typename Other>
|
||||
explicit AggregationMethodOneNumber(const Other & other) : data(other.data)
|
||||
{
|
||||
@ -225,6 +229,8 @@ struct AggregationMethodString
|
||||
{
|
||||
}
|
||||
|
||||
explicit AggregationMethodString(size_t size_hint) : data(size_hint) { }
|
||||
|
||||
using State = ColumnsHashing::HashMethodString<typename Data::value_type, Mapped>;
|
||||
|
||||
static const bool low_cardinality_optimization = false;
|
||||
@ -250,6 +256,8 @@ struct AggregationMethodStringNoCache
|
||||
|
||||
AggregationMethodStringNoCache() = default;
|
||||
|
||||
explicit AggregationMethodStringNoCache(size_t size_hint) : data(size_hint) { }
|
||||
|
||||
template <typename Other>
|
||||
explicit AggregationMethodStringNoCache(const Other & other) : data(other.data)
|
||||
{
|
||||
@ -280,6 +288,8 @@ struct AggregationMethodFixedString
|
||||
|
||||
AggregationMethodFixedString() = default;
|
||||
|
||||
explicit AggregationMethodFixedString(size_t size_hint) : data(size_hint) { }
|
||||
|
||||
template <typename Other>
|
||||
explicit AggregationMethodFixedString(const Other & other) : data(other.data)
|
||||
{
|
||||
@ -309,6 +319,8 @@ struct AggregationMethodFixedStringNoCache
|
||||
|
||||
AggregationMethodFixedStringNoCache() = default;
|
||||
|
||||
explicit AggregationMethodFixedStringNoCache(size_t size_hint) : data(size_hint) { }
|
||||
|
||||
template <typename Other>
|
||||
explicit AggregationMethodFixedStringNoCache(const Other & other) : data(other.data)
|
||||
{
|
||||
@ -382,6 +394,8 @@ struct AggregationMethodKeysFixed
|
||||
|
||||
AggregationMethodKeysFixed() = default;
|
||||
|
||||
explicit AggregationMethodKeysFixed(size_t size_hint) : data(size_hint) { }
|
||||
|
||||
template <typename Other>
|
||||
explicit AggregationMethodKeysFixed(const Other & other) : data(other.data)
|
||||
{
|
||||
@ -473,6 +487,8 @@ struct AggregationMethodSerialized
|
||||
|
||||
AggregationMethodSerialized() = default;
|
||||
|
||||
explicit AggregationMethodSerialized(size_t size_hint) : data(size_hint) { }
|
||||
|
||||
template <typename Other>
|
||||
explicit AggregationMethodSerialized(const Other & other) : data(other.data)
|
||||
{
|
||||
@ -652,21 +668,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
|
||||
~AggregatedDataVariants();
|
||||
|
||||
void init(Type type_)
|
||||
{
|
||||
switch (type_)
|
||||
{
|
||||
case Type::EMPTY: break;
|
||||
case Type::without_key: break;
|
||||
|
||||
#define M(NAME, IS_TWO_LEVEL) \
|
||||
case Type::NAME: (NAME) = std::make_unique<decltype(NAME)::element_type>(); break;
|
||||
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
||||
#undef M
|
||||
}
|
||||
|
||||
type = type_;
|
||||
}
|
||||
void init(Type type_, std::optional<size_t> size_hint = std::nullopt);
|
||||
|
||||
/// Number of rows (different keys).
|
||||
size_t size() const
|
||||
@ -929,29 +931,61 @@ public:
|
||||
bool compile_aggregate_expressions;
|
||||
size_t min_count_to_compile_aggregate_expression;
|
||||
|
||||
struct StatsCollectingParams
|
||||
{
|
||||
StatsCollectingParams();
|
||||
|
||||
StatsCollectingParams(
|
||||
const ASTPtr & select_query_,
|
||||
bool collect_hash_table_stats_during_aggregation_,
|
||||
size_t max_entries_for_hash_table_stats_,
|
||||
size_t max_size_to_preallocate_for_aggregation_);
|
||||
|
||||
bool isCollectionAndUseEnabled() const;
|
||||
|
||||
const UInt64 key = 0;
|
||||
const size_t max_entries_for_hash_table_stats = 0;
|
||||
const size_t max_size_to_preallocate_for_aggregation = 0;
|
||||
};
|
||||
StatsCollectingParams stats_collecting_params;
|
||||
|
||||
Params(
|
||||
const Block & src_header_,
|
||||
const ColumnNumbers & keys_, const AggregateDescriptions & aggregates_,
|
||||
bool overflow_row_, size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_,
|
||||
size_t group_by_two_level_threshold_, size_t group_by_two_level_threshold_bytes_,
|
||||
const ColumnNumbers & keys_,
|
||||
const AggregateDescriptions & aggregates_,
|
||||
bool overflow_row_,
|
||||
size_t max_rows_to_group_by_,
|
||||
OverflowMode group_by_overflow_mode_,
|
||||
size_t group_by_two_level_threshold_,
|
||||
size_t group_by_two_level_threshold_bytes_,
|
||||
size_t max_bytes_before_external_group_by_,
|
||||
bool empty_result_for_aggregation_by_empty_set_,
|
||||
VolumePtr tmp_volume_, size_t max_threads_,
|
||||
VolumePtr tmp_volume_,
|
||||
size_t max_threads_,
|
||||
size_t min_free_disk_space_,
|
||||
bool compile_aggregate_expressions_,
|
||||
size_t min_count_to_compile_aggregate_expression_,
|
||||
const Block & intermediate_header_ = {})
|
||||
: src_header(src_header_),
|
||||
intermediate_header(intermediate_header_),
|
||||
keys(keys_), aggregates(aggregates_), keys_size(keys.size()), aggregates_size(aggregates.size()),
|
||||
overflow_row(overflow_row_), max_rows_to_group_by(max_rows_to_group_by_), group_by_overflow_mode(group_by_overflow_mode_),
|
||||
group_by_two_level_threshold(group_by_two_level_threshold_), group_by_two_level_threshold_bytes(group_by_two_level_threshold_bytes_),
|
||||
max_bytes_before_external_group_by(max_bytes_before_external_group_by_),
|
||||
empty_result_for_aggregation_by_empty_set(empty_result_for_aggregation_by_empty_set_),
|
||||
tmp_volume(tmp_volume_), max_threads(max_threads_),
|
||||
min_free_disk_space(min_free_disk_space_),
|
||||
compile_aggregate_expressions(compile_aggregate_expressions_),
|
||||
min_count_to_compile_aggregate_expression(min_count_to_compile_aggregate_expression_)
|
||||
const Block & intermediate_header_ = {},
|
||||
const StatsCollectingParams & stats_collecting_params_ = {})
|
||||
: src_header(src_header_)
|
||||
, intermediate_header(intermediate_header_)
|
||||
, keys(keys_)
|
||||
, aggregates(aggregates_)
|
||||
, keys_size(keys.size())
|
||||
, aggregates_size(aggregates.size())
|
||||
, overflow_row(overflow_row_)
|
||||
, max_rows_to_group_by(max_rows_to_group_by_)
|
||||
, group_by_overflow_mode(group_by_overflow_mode_)
|
||||
, group_by_two_level_threshold(group_by_two_level_threshold_)
|
||||
, group_by_two_level_threshold_bytes(group_by_two_level_threshold_bytes_)
|
||||
, max_bytes_before_external_group_by(max_bytes_before_external_group_by_)
|
||||
, empty_result_for_aggregation_by_empty_set(empty_result_for_aggregation_by_empty_set_)
|
||||
, tmp_volume(tmp_volume_)
|
||||
, max_threads(max_threads_)
|
||||
, min_free_disk_space(min_free_disk_space_)
|
||||
, compile_aggregate_expressions(compile_aggregate_expressions_)
|
||||
, min_count_to_compile_aggregate_expression(min_count_to_compile_aggregate_expression_)
|
||||
, stats_collecting_params(stats_collecting_params_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -1350,4 +1384,13 @@ APPLY_FOR_AGGREGATED_VARIANTS(M)
|
||||
|
||||
#undef M
|
||||
|
||||
|
||||
struct HashTablesCacheStatistics
|
||||
{
|
||||
size_t entries = 0;
|
||||
size_t hits = 0;
|
||||
size_t misses = 0;
|
||||
};
|
||||
|
||||
std::optional<HashTablesCacheStatistics> getHashTablesCacheStatistics();
|
||||
}
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Interpreters/Aggregator.h>
|
||||
#include <Interpreters/AsynchronousMetrics.h>
|
||||
#include <Interpreters/AsynchronousMetricLog.h>
|
||||
#include <Interpreters/JIT/CompiledExpressionCache.h>
|
||||
@ -630,6 +631,15 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti
|
||||
|
||||
new_values["Uptime"] = getContext()->getUptimeSeconds();
|
||||
|
||||
{
|
||||
if (const auto stats = getHashTablesCacheStatistics())
|
||||
{
|
||||
new_values["HashTableStatsCacheEntries"] = stats->entries;
|
||||
new_values["HashTableStatsCacheHits"] = stats->hits;
|
||||
new_values["HashTableStatsCacheMisses"] = stats->misses;
|
||||
}
|
||||
}
|
||||
|
||||
/// Process process memory usage according to OS
|
||||
#if defined(OS_LINUX) || defined(OS_FREEBSD)
|
||||
{
|
||||
|
@ -169,6 +169,7 @@ public:
|
||||
if (columns.size() != float_features_count + cat_features_count)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Number of columns is different with number of features: columns size {} float features size {} + cat features size {}",
|
||||
columns.size(),
|
||||
float_features_count,
|
||||
cat_features_count);
|
||||
|
||||
|
@ -233,7 +233,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
|
||||
{
|
||||
assert(!db_and_table.first && !db_and_table.second);
|
||||
if (exception)
|
||||
exception->emplace(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs());
|
||||
exception->emplace(fmt::format("Table {} doesn't exist", table_id.getNameForLogs()), ErrorCodes::UNKNOWN_TABLE);
|
||||
return {};
|
||||
}
|
||||
|
||||
@ -263,7 +263,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
|
||||
/// If table_id has no UUID, then the name of database was specified by user and table_id was not resolved through context.
|
||||
/// Do not allow access to TEMPORARY_DATABASE because it contains all temporary tables of all contexts and users.
|
||||
if (exception)
|
||||
exception->emplace(ErrorCodes::DATABASE_ACCESS_DENIED, "Direct access to `{}` database is not allowed", String(TEMPORARY_DATABASE));
|
||||
exception->emplace(fmt::format("Direct access to `{}` database is not allowed", TEMPORARY_DATABASE), ErrorCodes::DATABASE_ACCESS_DENIED);
|
||||
return {};
|
||||
}
|
||||
|
||||
@ -274,7 +274,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
|
||||
if (databases.end() == it)
|
||||
{
|
||||
if (exception)
|
||||
exception->emplace(ErrorCodes::UNKNOWN_DATABASE, "Database {} doesn't exist", backQuoteIfNeed(table_id.getDatabaseName()));
|
||||
exception->emplace(fmt::format("Database {} doesn't exist", backQuoteIfNeed(table_id.getDatabaseName())), ErrorCodes::UNKNOWN_DATABASE);
|
||||
return {};
|
||||
}
|
||||
database = it->second;
|
||||
@ -282,7 +282,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
|
||||
|
||||
auto table = database->tryGetTable(table_id.table_name, context_);
|
||||
if (!table && exception)
|
||||
exception->emplace(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs());
|
||||
exception->emplace(fmt::format("Table {} doesn't exist", table_id.getNameForLogs()), ErrorCodes::UNKNOWN_TABLE);
|
||||
if (!table)
|
||||
database = nullptr;
|
||||
|
||||
|
@ -360,6 +360,7 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
|
||||
auto new_context = Context::createCopy(context);
|
||||
new_context->setSettings(new_settings);
|
||||
new_context->setInsertionTable(getContext()->getInsertionTable());
|
||||
|
||||
InterpreterSelectWithUnionQuery interpreter_select{
|
||||
query.select, new_context, SelectQueryOptions(QueryProcessingStage::Complete, 1)};
|
||||
|
@ -2098,6 +2098,12 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac
|
||||
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
const auto stats_collecting_params = Aggregator::Params::StatsCollectingParams(
|
||||
query_ptr,
|
||||
settings.collect_hash_table_stats_during_aggregation,
|
||||
settings.max_entries_for_hash_table_stats,
|
||||
settings.max_size_to_preallocate_for_aggregation);
|
||||
|
||||
Aggregator::Params params(
|
||||
header_before_aggregation,
|
||||
keys,
|
||||
@ -2115,7 +2121,9 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac
|
||||
settings.max_threads,
|
||||
settings.min_free_disk_space_for_temporary_data,
|
||||
settings.compile_aggregate_expressions,
|
||||
settings.min_count_to_compile_aggregate_expression);
|
||||
settings.min_count_to_compile_aggregate_expression,
|
||||
Block{},
|
||||
stats_collecting_params);
|
||||
|
||||
SortDescription group_by_sort_description;
|
||||
|
||||
|
@ -323,12 +323,13 @@ Chunk DDLQueryStatusSource::generate()
|
||||
if (throw_on_timeout)
|
||||
{
|
||||
if (!first_exception)
|
||||
first_exception = std::make_unique<Exception>(ErrorCodes::TIMEOUT_EXCEEDED, msg_format,
|
||||
node_path, timeout_seconds, num_unfinished_hosts, num_active_hosts);
|
||||
first_exception = std::make_unique<Exception>(
|
||||
fmt::format(msg_format, node_path, timeout_seconds, num_unfinished_hosts, num_active_hosts),
|
||||
ErrorCodes::TIMEOUT_EXCEEDED);
|
||||
return {};
|
||||
}
|
||||
|
||||
LOG_INFO(log, fmt::runtime(msg_format), node_path, timeout_seconds, num_unfinished_hosts, num_active_hosts);
|
||||
LOG_INFO(log, msg_format, node_path, timeout_seconds, num_unfinished_hosts, num_active_hosts);
|
||||
|
||||
NameSet unfinished_hosts = waiting_hosts;
|
||||
for (const auto & host_id : finished_hosts)
|
||||
@ -361,9 +362,12 @@ Chunk DDLQueryStatusSource::generate()
|
||||
/// Paradoxically, this exception will be throw even in case of "never_throw" mode.
|
||||
|
||||
if (!first_exception)
|
||||
first_exception = std::make_unique<Exception>(ErrorCodes::UNFINISHED,
|
||||
"Cannot provide query execution status. The query's node {} has been deleted by the cleaner"
|
||||
" since it was finished (or its lifetime is expired)", node_path);
|
||||
first_exception = std::make_unique<Exception>(
|
||||
fmt::format(
|
||||
"Cannot provide query execution status. The query's node {} has been deleted by the cleaner"
|
||||
" since it was finished (or its lifetime is expired)",
|
||||
node_path),
|
||||
ErrorCodes::UNFINISHED);
|
||||
return {};
|
||||
}
|
||||
|
||||
@ -389,7 +393,8 @@ Chunk DDLQueryStatusSource::generate()
|
||||
if (status.code != 0 && !first_exception
|
||||
&& context->getSettingsRef().distributed_ddl_output_mode != DistributedDDLOutputMode::NEVER_THROW)
|
||||
{
|
||||
first_exception = std::make_unique<Exception>(status.code, "There was an error on [{}:{}]: {}", host, port, status.message);
|
||||
first_exception = std::make_unique<Exception>(
|
||||
fmt::format("There was an error on [{}:{}]: {}", host, port, status.message), status.code);
|
||||
}
|
||||
|
||||
++num_hosts_finished;
|
||||
|
@ -683,6 +683,14 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode);
|
||||
}
|
||||
|
||||
if (const auto * insert_interpreter = typeid_cast<const InterpreterInsertQuery *>(&*interpreter))
|
||||
{
|
||||
/// Save insertion table (not table function). TODO: support remote() table function.
|
||||
auto table_id = insert_interpreter->getDatabaseTable();
|
||||
if (!table_id.empty())
|
||||
context->setInsertionTable(std::move(table_id));
|
||||
}
|
||||
|
||||
{
|
||||
std::unique_ptr<OpenTelemetrySpanHolder> span;
|
||||
if (context->query_trace_context.trace_id != UUID())
|
||||
@ -693,14 +701,6 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
}
|
||||
res = interpreter->execute();
|
||||
}
|
||||
|
||||
if (const auto * insert_interpreter = typeid_cast<const InterpreterInsertQuery *>(&*interpreter))
|
||||
{
|
||||
/// Save insertion table (not table function). TODO: support remote() table function.
|
||||
auto table_id = insert_interpreter->getDatabaseTable();
|
||||
if (!table_id.empty())
|
||||
context->setInsertionTable(std::move(table_id));
|
||||
}
|
||||
}
|
||||
|
||||
if (process_list_entry)
|
||||
|
@ -9,30 +9,31 @@ namespace DB
|
||||
|
||||
CallbackRunner threadPoolCallbackRunner(ThreadPool & pool)
|
||||
{
|
||||
return [pool = &pool, thread_group = CurrentThread::getGroup()](auto callback)
|
||||
return [pool = &pool, thread_group = CurrentThread::getGroup()](auto callback) mutable
|
||||
{
|
||||
pool->scheduleOrThrow([callback = std::move(callback), thread_group]()
|
||||
{
|
||||
if (thread_group)
|
||||
CurrentThread::attachTo(thread_group);
|
||||
|
||||
SCOPE_EXIT_SAFE({
|
||||
pool->scheduleOrThrow(
|
||||
[&, callback = std::move(callback), thread_group]()
|
||||
{
|
||||
if (thread_group)
|
||||
CurrentThread::detachQueryIfNotDetached();
|
||||
CurrentThread::attachTo(thread_group);
|
||||
|
||||
/// After we detached from the thread_group, parent for memory_tracker inside ThreadStatus will be reset to it's parent.
|
||||
/// Typically, it may be changes from Process to User.
|
||||
/// Usually it could be ok, because thread pool task is executed before user-level memory tracker is destroyed.
|
||||
/// However, thread could stay alive inside the thread pool, and it's ThreadStatus as well.
|
||||
/// When, finally, we destroy the thread (and the ThreadStatus),
|
||||
/// it can use memory tracker in the ~ThreadStatus in order to alloc/free untracked_memory,
|
||||
/// and by this time user-level memory tracker may be already destroyed.
|
||||
///
|
||||
/// As a work-around, reset memory tracker to total, which is always alive.
|
||||
CurrentThread::get().memory_tracker.setParent(&total_memory_tracker);
|
||||
SCOPE_EXIT_SAFE({
|
||||
if (thread_group)
|
||||
CurrentThread::detachQueryIfNotDetached();
|
||||
|
||||
/// After we detached from the thread_group, parent for memory_tracker inside ThreadStatus will be reset to it's parent.
|
||||
/// Typically, it may be changes from Process to User.
|
||||
/// Usually it could be ok, because thread pool task is executed before user-level memory tracker is destroyed.
|
||||
/// However, thread could stay alive inside the thread pool, and it's ThreadStatus as well.
|
||||
/// When, finally, we destroy the thread (and the ThreadStatus),
|
||||
/// it can use memory tracker in the ~ThreadStatus in order to alloc/free untracked_memory,
|
||||
/// and by this time user-level memory tracker may be already destroyed.
|
||||
///
|
||||
/// As a work-around, reset memory tracker to total, which is always alive.
|
||||
CurrentThread::get().memory_tracker.setParent(&total_memory_tracker);
|
||||
});
|
||||
callback();
|
||||
});
|
||||
callback();
|
||||
});
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -328,14 +328,20 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node
|
||||
|
||||
ASTPtr elem;
|
||||
SubqueryFunctionType subquery_function_type = SubqueryFunctionType::NONE;
|
||||
if (allow_any_all_operators && ParserKeyword("ANY").ignore(pos, expected))
|
||||
subquery_function_type = SubqueryFunctionType::ANY;
|
||||
else if (allow_any_all_operators && ParserKeyword("ALL").ignore(pos, expected))
|
||||
subquery_function_type = SubqueryFunctionType::ALL;
|
||||
else if (!(remaining_elem_parser ? remaining_elem_parser : first_elem_parser)->parse(pos, elem, expected))
|
||||
return false;
|
||||
|
||||
if (comparison_expression)
|
||||
{
|
||||
if (ParserKeyword("ANY").ignore(pos, expected))
|
||||
subquery_function_type = SubqueryFunctionType::ANY;
|
||||
else if (ParserKeyword("ALL").ignore(pos, expected))
|
||||
subquery_function_type = SubqueryFunctionType::ALL;
|
||||
}
|
||||
|
||||
if (subquery_function_type != SubqueryFunctionType::NONE && !ParserSubquery().parse(pos, elem, expected))
|
||||
subquery_function_type = SubqueryFunctionType::NONE;
|
||||
|
||||
if (subquery_function_type == SubqueryFunctionType::NONE
|
||||
&& !(remaining_elem_parser ? remaining_elem_parser : first_elem_parser)->parse(pos, elem, expected))
|
||||
return false;
|
||||
|
||||
/// the first argument of the function is the previous element, the second is the next one
|
||||
@ -346,7 +352,7 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node
|
||||
exp_list->children.push_back(node);
|
||||
exp_list->children.push_back(elem);
|
||||
|
||||
if (allow_any_all_operators && subquery_function_type != SubqueryFunctionType::NONE && !modifyAST(function, subquery_function_type))
|
||||
if (comparison_expression && subquery_function_type != SubqueryFunctionType::NONE && !modifyAST(function, subquery_function_type))
|
||||
return false;
|
||||
|
||||
/** special exception for the access operator to the element of the array `x[y]`, which
|
||||
|
@ -122,7 +122,7 @@ private:
|
||||
ParserPtr first_elem_parser;
|
||||
ParserPtr remaining_elem_parser;
|
||||
/// =, !=, <, > ALL (subquery) / ANY (subquery)
|
||||
bool allow_any_all_operators = false;
|
||||
bool comparison_expression = false;
|
||||
|
||||
public:
|
||||
/** `operators_` - allowed operators and their corresponding functions
|
||||
@ -133,9 +133,9 @@ public:
|
||||
}
|
||||
|
||||
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_,
|
||||
Operators_t overlapping_operators_to_skip_, ParserPtr && first_elem_parser_, bool allow_any_all_operators_ = false)
|
||||
Operators_t overlapping_operators_to_skip_, ParserPtr && first_elem_parser_, bool comparison_expression_ = false)
|
||||
: operators(operators_), overlapping_operators_to_skip(overlapping_operators_to_skip_),
|
||||
first_elem_parser(std::move(first_elem_parser_)), allow_any_all_operators(allow_any_all_operators_)
|
||||
first_elem_parser(std::move(first_elem_parser_)), comparison_expression(comparison_expression_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -18,6 +18,10 @@ public:
|
||||
|
||||
virtual NamesAndTypesList readSchema() = 0;
|
||||
|
||||
/// True if order of columns is important in format.
|
||||
/// Exceptions: JSON, TSKV.
|
||||
virtual bool hasStrictOrderOfColumns() const { return true; }
|
||||
|
||||
virtual ~ISchemaReader() = default;
|
||||
|
||||
protected:
|
||||
@ -60,6 +64,7 @@ class IRowWithNamesSchemaReader : public ISchemaReader
|
||||
public:
|
||||
IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr);
|
||||
NamesAndTypesList readSchema() override;
|
||||
bool hasStrictOrderOfColumns() const override { return false; }
|
||||
|
||||
protected:
|
||||
/// Read one row and determine types of columns in it.
|
||||
|
@ -359,7 +359,7 @@ bool MsgPackVisitor::visit_ext(const char * value, uint32_t size)
|
||||
return true;
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported MsgPack extension type: {%x}", type);
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported MsgPack extension type: {:x}", type);
|
||||
}
|
||||
|
||||
void MsgPackVisitor::parse_error(size_t, size_t) // NOLINT
|
||||
@ -498,7 +498,7 @@ DataTypePtr MsgPackSchemaReader::getDataType(const msgpack::object & object)
|
||||
msgpack::object_ext object_ext = object.via.ext;
|
||||
if (object_ext.type() == int8_t(MsgPackExtensionTypes::UUIDType))
|
||||
return std::make_shared<DataTypeUUID>();
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Msgpack extension type {%x} is not supported", object_ext.type());
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Msgpack extension type {:x} is not supported", object_ext.type());
|
||||
}
|
||||
}
|
||||
__builtin_unreachable();
|
||||
|
@ -45,7 +45,8 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
static MergeTreeReaderSettings getMergeTreeReaderSettings(const ContextPtr & context)
|
||||
static MergeTreeReaderSettings getMergeTreeReaderSettings(
|
||||
const ContextPtr & context, const SelectQueryInfo & query_info)
|
||||
{
|
||||
const auto & settings = context->getSettingsRef();
|
||||
return
|
||||
@ -53,6 +54,7 @@ static MergeTreeReaderSettings getMergeTreeReaderSettings(const ContextPtr & con
|
||||
.read_settings = context->getReadSettings(),
|
||||
.save_marks_in_cache = true,
|
||||
.checksum_on_read = settings.checksum_on_read,
|
||||
.read_in_order = query_info.input_order_info != nullptr,
|
||||
};
|
||||
}
|
||||
|
||||
@ -82,7 +84,7 @@ ReadFromMergeTree::ReadFromMergeTree(
|
||||
getPrewhereInfo(query_info_),
|
||||
data_.getPartitionValueType(),
|
||||
virt_column_names_)})
|
||||
, reader_settings(getMergeTreeReaderSettings(context_))
|
||||
, reader_settings(getMergeTreeReaderSettings(context_, query_info_))
|
||||
, prepared_parts(std::move(parts_))
|
||||
, real_column_names(std::move(real_column_names_))
|
||||
, virt_column_names(std::move(virt_column_names_))
|
||||
@ -206,6 +208,7 @@ ProcessorPtr ReadFromMergeTree::createSource(
|
||||
.colums_to_read = required_columns
|
||||
};
|
||||
}
|
||||
|
||||
return std::make_shared<TSource>(
|
||||
data, storage_snapshot, part.data_part, max_block_size, preferred_block_size_bytes,
|
||||
preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, prewhere_info,
|
||||
@ -921,7 +924,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
|
||||
total_marks_pk += part->index_granularity.getMarksCountWithoutFinal();
|
||||
parts_before_pk = parts.size();
|
||||
|
||||
auto reader_settings = getMergeTreeReaderSettings(context);
|
||||
auto reader_settings = getMergeTreeReaderSettings(context, query_info);
|
||||
|
||||
bool use_skip_indexes = settings.use_skip_indexes;
|
||||
if (select.final() && !settings.use_skip_indexes_if_final)
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Poco/URI.h>
|
||||
#include <boost/algorithm/string/replace.hpp>
|
||||
#include <re2/re2.h>
|
||||
#include <filesystem>
|
||||
|
||||
#if USE_HDFS
|
||||
#include <Common/ShellCommand.h>
|
||||
@ -25,14 +26,14 @@ const String HDFSBuilderWrapper::CONFIG_PREFIX = "hdfs";
|
||||
const String HDFS_URL_REGEXP = "^hdfs://[^/]*/.*";
|
||||
|
||||
void HDFSBuilderWrapper::loadFromConfig(const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_path, bool isUser)
|
||||
const String & prefix, bool isUser)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
|
||||
config.keys(config_path, keys);
|
||||
config.keys(prefix, keys);
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
const String key_path = config_path + "." + key;
|
||||
const String key_path = prefix + "." + key;
|
||||
|
||||
String key_name;
|
||||
if (key == "hadoop_kerberos_keytab")
|
||||
@ -122,9 +123,17 @@ HDFSBuilderWrapper createHDFSBuilder(const String & uri_str, const Poco::Util::A
|
||||
throw Exception("Illegal HDFS URI: " + uri.toString(), ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
// Shall set env LIBHDFS3_CONF *before* HDFSBuilderWrapper construction.
|
||||
const String & libhdfs3_conf = config.getString(HDFSBuilderWrapper::CONFIG_PREFIX + ".libhdfs3_conf", "");
|
||||
String libhdfs3_conf = config.getString(HDFSBuilderWrapper::CONFIG_PREFIX + ".libhdfs3_conf", "");
|
||||
if (!libhdfs3_conf.empty())
|
||||
{
|
||||
if (std::filesystem::path{libhdfs3_conf}.is_relative() && !std::filesystem::exists(libhdfs3_conf))
|
||||
{
|
||||
const String config_path = config.getString("config-file", "config.xml");
|
||||
const auto config_dir = std::filesystem::path{config_path}.remove_filename();
|
||||
if (std::filesystem::exists(config_dir / libhdfs3_conf))
|
||||
libhdfs3_conf = std::filesystem::absolute(config_dir / libhdfs3_conf);
|
||||
}
|
||||
|
||||
setenv("LIBHDFS3_CONF", libhdfs3_conf.c_str(), 1);
|
||||
}
|
||||
HDFSBuilderWrapper builder;
|
||||
|
@ -67,7 +67,7 @@ public:
|
||||
hdfsBuilder * get() { return hdfs_builder; }
|
||||
|
||||
private:
|
||||
void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_path, bool isUser = false);
|
||||
void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const String & prefix, bool isUser = false);
|
||||
|
||||
String getKinitCmd();
|
||||
|
||||
|
@ -289,6 +289,7 @@ StorageHive::StorageHive(
|
||||
, partition_by_ast(partition_by_ast_)
|
||||
, storage_settings(std::move(storage_settings_))
|
||||
{
|
||||
/// Check hive metastore url.
|
||||
getContext()->getRemoteHostFilter().checkURL(Poco::URI(hive_metastore_url));
|
||||
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
@ -304,11 +305,13 @@ void StorageHive::lazyInitialize()
|
||||
if (has_initialized)
|
||||
return;
|
||||
|
||||
|
||||
auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext());
|
||||
auto hive_table_metadata = hive_metastore_client->getHiveTable(hive_database, hive_table);
|
||||
|
||||
hdfs_namenode_url = getNameNodeUrl(hive_table_metadata->sd.location);
|
||||
/// Check HDFS namenode url.
|
||||
getContext()->getRemoteHostFilter().checkURL(Poco::URI(hdfs_namenode_url));
|
||||
|
||||
table_schema = hive_table_metadata->sd.cols;
|
||||
|
||||
FileFormat hdfs_file_format = IHiveFile::toFileFormat(hive_table_metadata->sd.inputFormat);
|
||||
@ -319,10 +322,8 @@ void StorageHive::lazyInitialize()
|
||||
format_name = "HiveText";
|
||||
break;
|
||||
case FileFormat::RC_FILE:
|
||||
/// TODO to be implemented
|
||||
throw Exception("Unsopported hive format rc_file", ErrorCodes::NOT_IMPLEMENTED);
|
||||
case FileFormat::SEQUENCE_FILE:
|
||||
/// TODO to be implemented
|
||||
throw Exception("Unsopported hive format sequence_file", ErrorCodes::NOT_IMPLEMENTED);
|
||||
case FileFormat::AVRO:
|
||||
format_name = "Avro";
|
||||
|
@ -1,10 +1,13 @@
|
||||
// Needs to go first because its partial specialization of fmt::formatter
|
||||
// should be defined before any instantiation
|
||||
#include <fmt/ostream.h>
|
||||
|
||||
#include <Storages/Kafka/ReadBufferFromKafkaConsumer.h>
|
||||
|
||||
#include <base/logger_useful.h>
|
||||
|
||||
#include <cppkafka/cppkafka.h>
|
||||
#include <boost/algorithm/string/join.hpp>
|
||||
#include <fmt/ostream.h>
|
||||
#include <algorithm>
|
||||
|
||||
namespace DB
|
||||
|
@ -583,9 +583,10 @@ size_t IMergeTreeDataPart::getFileSizeOrZero(const String & file_name) const
|
||||
return checksum->second.file_size;
|
||||
}
|
||||
|
||||
String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize(const StorageMetadataPtr & metadata_snapshot) const
|
||||
String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize(const StorageSnapshotPtr & storage_snapshot) const
|
||||
{
|
||||
const auto & storage_columns = metadata_snapshot->getColumns().getAllPhysical();
|
||||
auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects().withSubcolumns();
|
||||
auto storage_columns = storage_snapshot->getColumns(options);
|
||||
MergeTreeData::AlterConversions alter_conversions;
|
||||
if (!parent_part)
|
||||
alter_conversions = storage.getAlterConversionsForPart(shared_from_this());
|
||||
|
@ -170,7 +170,7 @@ public:
|
||||
|
||||
/// Returns the name of a column with minimum compressed size (as returned by getColumnSize()).
|
||||
/// If no checksums are present returns the name of the first physically existing column.
|
||||
String getColumnNameWithMinimumCompressedSize(const StorageMetadataPtr & metadata_snapshot) const;
|
||||
String getColumnNameWithMinimumCompressedSize(const StorageSnapshotPtr & storage_snapshot) const;
|
||||
|
||||
bool contains(const IMergeTreeDataPart & other) const { return info.contains(other.info); }
|
||||
|
||||
|
@ -24,7 +24,7 @@ namespace
|
||||
/// least one existing (physical) column in part.
|
||||
bool injectRequiredColumnsRecursively(
|
||||
const String & column_name,
|
||||
const ColumnsDescription & storage_columns,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
const MergeTreeData::AlterConversions & alter_conversions,
|
||||
const MergeTreeData::DataPartPtr & part,
|
||||
Names & columns,
|
||||
@ -36,7 +36,8 @@ bool injectRequiredColumnsRecursively(
|
||||
/// stages.
|
||||
checkStackSize();
|
||||
|
||||
auto column_in_storage = storage_columns.tryGetColumnOrSubcolumn(GetColumnsOptions::AllPhysical, column_name);
|
||||
auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withSubcolumns().withExtendedObjects();
|
||||
auto column_in_storage = storage_snapshot->tryGetColumn(options, column_name);
|
||||
if (column_in_storage)
|
||||
{
|
||||
auto column_name_in_part = column_in_storage->getNameInStorage();
|
||||
@ -63,7 +64,8 @@ bool injectRequiredColumnsRecursively(
|
||||
|
||||
/// Column doesn't have default value and don't exist in part
|
||||
/// don't need to add to required set.
|
||||
const auto column_default = storage_columns.getDefault(column_name);
|
||||
auto metadata_snapshot = storage_snapshot->getMetadataForQuery();
|
||||
const auto column_default = metadata_snapshot->getColumns().getDefault(column_name);
|
||||
if (!column_default)
|
||||
return false;
|
||||
|
||||
@ -73,39 +75,36 @@ bool injectRequiredColumnsRecursively(
|
||||
|
||||
bool result = false;
|
||||
for (const auto & identifier : identifiers)
|
||||
result |= injectRequiredColumnsRecursively(identifier, storage_columns, alter_conversions, part, columns, required_columns, injected_columns);
|
||||
result |= injectRequiredColumnsRecursively(identifier, storage_snapshot, alter_conversions, part, columns, required_columns, injected_columns);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData::DataPartPtr & part, Names & columns)
|
||||
NameSet injectRequiredColumns(
|
||||
const MergeTreeData & storage,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
const MergeTreeData::DataPartPtr & part,
|
||||
Names & columns)
|
||||
{
|
||||
NameSet required_columns{std::begin(columns), std::end(columns)};
|
||||
NameSet injected_columns;
|
||||
|
||||
bool have_at_least_one_physical_column = false;
|
||||
|
||||
const auto & storage_columns = metadata_snapshot->getColumns();
|
||||
MergeTreeData::AlterConversions alter_conversions;
|
||||
if (!part->isProjectionPart())
|
||||
alter_conversions = storage.getAlterConversionsForPart(part);
|
||||
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
auto name_in_storage = Nested::extractTableName(columns[i]);
|
||||
if (storage_columns.has(name_in_storage) && isObject(storage_columns.get(name_in_storage).type))
|
||||
{
|
||||
have_at_least_one_physical_column = true;
|
||||
continue;
|
||||
}
|
||||
|
||||
/// We are going to fetch only physical columns
|
||||
if (!storage_columns.hasColumnOrSubcolumn(GetColumnsOptions::AllPhysical, columns[i]))
|
||||
throw Exception("There is no physical column or subcolumn " + columns[i] + " in table.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||
auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withSubcolumns().withExtendedObjects();
|
||||
if (!storage_snapshot->tryGetColumn(options, columns[i]))
|
||||
throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no physical column or subcolumn {} in table", columns[i]);
|
||||
|
||||
have_at_least_one_physical_column |= injectRequiredColumnsRecursively(
|
||||
columns[i], storage_columns, alter_conversions,
|
||||
columns[i], storage_snapshot, alter_conversions,
|
||||
part, columns, required_columns, injected_columns);
|
||||
}
|
||||
|
||||
@ -115,7 +114,7 @@ NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageMetada
|
||||
*/
|
||||
if (!have_at_least_one_physical_column)
|
||||
{
|
||||
const auto minimum_size_column_name = part->getColumnNameWithMinimumCompressedSize(metadata_snapshot);
|
||||
const auto minimum_size_column_name = part->getColumnNameWithMinimumCompressedSize(storage_snapshot);
|
||||
columns.push_back(minimum_size_column_name);
|
||||
/// correctly report added column
|
||||
injected_columns.insert(columns.back());
|
||||
@ -271,7 +270,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
Names pre_column_names;
|
||||
|
||||
/// inject columns required for defaults evaluation
|
||||
bool should_reorder = !injectRequiredColumns(storage, storage_snapshot->getMetadataForQuery(), data_part, column_names).empty();
|
||||
bool should_reorder = !injectRequiredColumns(storage, storage_snapshot, data_part, column_names).empty();
|
||||
|
||||
if (prewhere_info)
|
||||
{
|
||||
@ -296,7 +295,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
if (pre_column_names.empty())
|
||||
pre_column_names.push_back(column_names[0]);
|
||||
|
||||
const auto injected_pre_columns = injectRequiredColumns(storage, storage_snapshot->getMetadataForQuery(), data_part, pre_column_names);
|
||||
const auto injected_pre_columns = injectRequiredColumns(storage, storage_snapshot, data_part, pre_column_names);
|
||||
if (!injected_pre_columns.empty())
|
||||
should_reorder = true;
|
||||
|
||||
|
@ -22,7 +22,7 @@ using MergeTreeBlockSizePredictorPtr = std::shared_ptr<MergeTreeBlockSizePredict
|
||||
* so that you can calculate the DEFAULT expression for these columns.
|
||||
* Adds them to the `columns`.
|
||||
*/
|
||||
NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData::DataPartPtr & part, Names & columns);
|
||||
NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageSnapshotPtr & storage_snapshot, const MergeTreeData::DataPartPtr & part, Names & columns);
|
||||
|
||||
|
||||
/// A batch of work for MergeTreeThreadSelectBlockInputStream
|
||||
|
@ -879,12 +879,22 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
|
||||
{
|
||||
std::atomic<size_t> total_rows{0};
|
||||
|
||||
/// Do not check number of read rows if we have reading
|
||||
/// in order of sorting key with limit.
|
||||
/// In general case, when there exists WHERE clause
|
||||
/// it's impossible to estimate number of rows precisely,
|
||||
/// because we can stop reading at any time.
|
||||
|
||||
SizeLimits limits;
|
||||
if (settings.read_overflow_mode == OverflowMode::THROW && settings.max_rows_to_read)
|
||||
if (settings.read_overflow_mode == OverflowMode::THROW
|
||||
&& settings.max_rows_to_read
|
||||
&& !query_info.input_order_info)
|
||||
limits = SizeLimits(settings.max_rows_to_read, 0, settings.read_overflow_mode);
|
||||
|
||||
SizeLimits leaf_limits;
|
||||
if (settings.read_overflow_mode_leaf == OverflowMode::THROW && settings.max_rows_to_read_leaf)
|
||||
if (settings.read_overflow_mode_leaf == OverflowMode::THROW
|
||||
&& settings.max_rows_to_read_leaf
|
||||
&& !query_info.input_order_info)
|
||||
leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, 0, settings.read_overflow_mode_leaf);
|
||||
|
||||
auto mark_cache = context->getIndexMarkCache();
|
||||
|
@ -20,6 +20,8 @@ struct MergeTreeReaderSettings
|
||||
bool save_marks_in_cache = false;
|
||||
/// Validate checksums on reading (should be always enabled in production).
|
||||
bool checksum_on_read = true;
|
||||
/// True if we read in order of sorting key.
|
||||
bool read_in_order = false;
|
||||
};
|
||||
|
||||
struct MergeTreeWriterSettings
|
||||
|
@ -39,9 +39,12 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor(
|
||||
{
|
||||
/// Actually it means that parallel reading from replicas enabled
|
||||
/// and we have to collaborate with initiator.
|
||||
/// In this case we won't set approximate rows, because it will be accounted multiple times
|
||||
if (!extension_.has_value())
|
||||
/// In this case we won't set approximate rows, because it will be accounted multiple times.
|
||||
/// Also do not count amount of read rows if we read in order of sorting key,
|
||||
/// because we don't know actual amount of read rows in case when limit is set.
|
||||
if (!extension_.has_value() && !reader_settings.read_in_order)
|
||||
addTotalRowsApprox(total_rows);
|
||||
|
||||
ordered_names = header_without_virtual_columns.getNames();
|
||||
}
|
||||
|
||||
|
@ -41,7 +41,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource(
|
||||
addTotalRowsApprox(data_part->rows_count);
|
||||
|
||||
/// Add columns because we don't want to read empty blocks
|
||||
injectRequiredColumns(storage, storage_snapshot->metadata, data_part, columns_to_read);
|
||||
injectRequiredColumns(storage, storage_snapshot, data_part, columns_to_read);
|
||||
NamesAndTypesList columns_for_reader;
|
||||
if (take_column_types_from_storage)
|
||||
{
|
||||
|
@ -1,4 +1,6 @@
|
||||
#include <Common/config.h>
|
||||
#include "IO/ParallelReadBuffer.h"
|
||||
#include "IO/IOThreadPool.h"
|
||||
#include "Parsers/ASTCreateQuery.h"
|
||||
|
||||
#if USE_AWS_S3
|
||||
@ -238,7 +240,8 @@ StorageS3Source::StorageS3Source(
|
||||
String compression_hint_,
|
||||
const std::shared_ptr<Aws::S3::S3Client> & client_,
|
||||
const String & bucket_,
|
||||
std::shared_ptr<IteratorWrapper> file_iterator_)
|
||||
std::shared_ptr<IteratorWrapper> file_iterator_,
|
||||
const size_t download_thread_num_)
|
||||
: SourceWithProgress(getHeader(sample_block_, need_path, need_file))
|
||||
, WithContext(context_)
|
||||
, name(std::move(name_))
|
||||
@ -254,6 +257,7 @@ StorageS3Source::StorageS3Source(
|
||||
, with_file_column(need_file)
|
||||
, with_path_column(need_path)
|
||||
, file_iterator(file_iterator_)
|
||||
, download_thread_num(download_thread_num_)
|
||||
{
|
||||
initialize();
|
||||
}
|
||||
@ -275,28 +279,54 @@ bool StorageS3Source::initialize()
|
||||
|
||||
file_path = fs::path(bucket) / current_key;
|
||||
|
||||
read_buf = wrapReadBufferWithCompressionMethod(
|
||||
std::make_unique<ReadBufferFromS3>(client, bucket, current_key, max_single_read_retries, getContext()->getReadSettings()),
|
||||
chooseCompressionMethod(current_key, compression_hint));
|
||||
read_buf = wrapReadBufferWithCompressionMethod(createS3ReadBuffer(current_key), chooseCompressionMethod(current_key, compression_hint));
|
||||
|
||||
auto input_format = getContext()->getInputFormat(format, *read_buf, sample_block, max_block_size, format_settings);
|
||||
QueryPipelineBuilder builder;
|
||||
builder.init(Pipe(input_format));
|
||||
|
||||
if (columns_desc.hasDefaults())
|
||||
{
|
||||
builder.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
return std::make_shared<AddingDefaultsTransform>(header, columns_desc, *input_format, getContext());
|
||||
});
|
||||
builder.addSimpleTransform(
|
||||
[&](const Block & header)
|
||||
{ return std::make_shared<AddingDefaultsTransform>(header, columns_desc, *input_format, getContext()); });
|
||||
}
|
||||
|
||||
pipeline = std::make_unique<QueryPipeline>(QueryPipelineBuilder::getPipeline(std::move(builder)));
|
||||
reader = std::make_unique<PullingPipelineExecutor>(*pipeline);
|
||||
|
||||
initialized = false;
|
||||
return true;
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBuffer> StorageS3Source::createS3ReadBuffer(const String & key)
|
||||
{
|
||||
const size_t object_size = DB::S3::getObjectSize(client, bucket, key, false);
|
||||
|
||||
auto download_buffer_size = getContext()->getSettings().max_download_buffer_size;
|
||||
const bool use_parallel_download = download_buffer_size > 0 && download_thread_num > 1;
|
||||
const bool object_too_small = object_size < download_thread_num * download_buffer_size;
|
||||
if (!use_parallel_download || object_too_small)
|
||||
{
|
||||
LOG_TRACE(log, "Downloading object of size {} from S3 in single thread", object_size);
|
||||
return std::make_unique<ReadBufferFromS3>(client, bucket, key, max_single_read_retries, getContext()->getReadSettings());
|
||||
}
|
||||
|
||||
assert(object_size > 0);
|
||||
|
||||
if (download_buffer_size < DBMS_DEFAULT_BUFFER_SIZE)
|
||||
{
|
||||
LOG_WARNING(log, "Downloading buffer {} bytes too small, set at least {} bytes", download_buffer_size, DBMS_DEFAULT_BUFFER_SIZE);
|
||||
download_buffer_size = DBMS_DEFAULT_BUFFER_SIZE;
|
||||
}
|
||||
|
||||
auto factory = std::make_unique<ReadBufferS3Factory>(
|
||||
client, bucket, key, download_buffer_size, object_size, max_single_read_retries, getContext()->getReadSettings());
|
||||
LOG_TRACE(
|
||||
log, "Downloading from S3 in {} threads. Object size: {}, Range size: {}.", download_thread_num, object_size, download_buffer_size);
|
||||
|
||||
return std::make_unique<ParallelReadBuffer>(std::move(factory), threadPoolCallbackRunner(IOThreadPool::get()), download_thread_num);
|
||||
}
|
||||
|
||||
String StorageS3Source::getName() const
|
||||
{
|
||||
return name;
|
||||
@ -376,16 +406,6 @@ static bool checkIfObjectExists(const std::shared_ptr<Aws::S3::S3Client> & clien
|
||||
return false;
|
||||
}
|
||||
|
||||
// TODO: common thread pool for IO must be used instead after PR #35150
|
||||
static ThreadPool & getThreadPoolStorageS3()
|
||||
{
|
||||
constexpr size_t pool_size = 100;
|
||||
constexpr size_t queue_size = 1000000;
|
||||
static ThreadPool pool(pool_size, pool_size, queue_size);
|
||||
return pool;
|
||||
}
|
||||
|
||||
|
||||
class StorageS3Sink : public SinkToStorage
|
||||
{
|
||||
public:
|
||||
@ -408,10 +428,20 @@ public:
|
||||
{
|
||||
write_buf = wrapWriteBufferWithCompressionMethod(
|
||||
std::make_unique<WriteBufferFromS3>(
|
||||
client, bucket, key, min_upload_part_size,
|
||||
upload_part_size_multiply_factor, upload_part_size_multiply_parts_count_threshold,
|
||||
max_single_part_upload_size, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, threadPoolCallbackRunner(getThreadPoolStorageS3())), compression_method, 3);
|
||||
writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, {}, format_settings);
|
||||
client,
|
||||
bucket,
|
||||
key,
|
||||
min_upload_part_size,
|
||||
upload_part_size_multiply_factor,
|
||||
upload_part_size_multiply_parts_count_threshold,
|
||||
max_single_part_upload_size,
|
||||
std::nullopt,
|
||||
DBMS_DEFAULT_BUFFER_SIZE,
|
||||
threadPoolCallbackRunner(IOThreadPool::get())),
|
||||
compression_method,
|
||||
3);
|
||||
writer
|
||||
= FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, {}, format_settings);
|
||||
}
|
||||
|
||||
String getName() const override { return "StorageS3Sink"; }
|
||||
@ -670,6 +700,7 @@ Pipe StorageS3::read(
|
||||
block_for_format = storage_snapshot->metadata->getSampleBlock();
|
||||
}
|
||||
|
||||
const size_t max_download_threads = local_context->getSettingsRef().max_download_threads;
|
||||
for (size_t i = 0; i < num_streams; ++i)
|
||||
{
|
||||
pipes.emplace_back(std::make_shared<StorageS3Source>(
|
||||
@ -686,7 +717,8 @@ Pipe StorageS3::read(
|
||||
compression_method,
|
||||
client_auth.client,
|
||||
client_auth.uri.bucket,
|
||||
iterator_wrapper));
|
||||
iterator_wrapper,
|
||||
max_download_threads));
|
||||
}
|
||||
auto pipe = Pipe::unitePipes(std::move(pipes));
|
||||
|
||||
|
@ -74,7 +74,8 @@ public:
|
||||
String compression_hint_,
|
||||
const std::shared_ptr<Aws::S3::S3Client> & client_,
|
||||
const String & bucket,
|
||||
std::shared_ptr<IteratorWrapper> file_iterator_);
|
||||
std::shared_ptr<IteratorWrapper> file_iterator_,
|
||||
size_t download_thread_num);
|
||||
|
||||
String getName() const override;
|
||||
|
||||
@ -101,13 +102,17 @@ private:
|
||||
std::unique_ptr<PullingPipelineExecutor> reader;
|
||||
/// onCancel and generate can be called concurrently
|
||||
std::mutex reader_mutex;
|
||||
bool initialized = false;
|
||||
bool with_file_column = false;
|
||||
bool with_path_column = false;
|
||||
std::shared_ptr<IteratorWrapper> file_iterator;
|
||||
size_t download_thread_num = 1;
|
||||
|
||||
Poco::Logger * log = &Poco::Logger::get("StorageS3Source");
|
||||
|
||||
/// Recreate ReadBuffer and BlockInputStream for each file.
|
||||
bool initialize();
|
||||
|
||||
std::unique_ptr<ReadBuffer> createS3ReadBuffer(const String & key);
|
||||
};
|
||||
|
||||
/**
|
||||
|
@ -51,40 +51,42 @@ NamesAndTypesList StorageSnapshot::getColumns(const GetColumnsOptions & options)
|
||||
NamesAndTypesList StorageSnapshot::getColumnsByNames(const GetColumnsOptions & options, const Names & names) const
|
||||
{
|
||||
NamesAndTypesList res;
|
||||
const auto & columns = getMetadataForQuery()->getColumns();
|
||||
for (const auto & name : names)
|
||||
res.push_back(getColumn(options, name));
|
||||
return res;
|
||||
}
|
||||
|
||||
std::optional<NameAndTypePair> StorageSnapshot::tryGetColumn(const GetColumnsOptions & options, const String & column_name) const
|
||||
{
|
||||
const auto & columns = getMetadataForQuery()->getColumns();
|
||||
auto column = columns.tryGetColumn(options, column_name);
|
||||
if (column && (!isObject(column->type) || !options.with_extended_objects))
|
||||
return column;
|
||||
|
||||
if (options.with_extended_objects)
|
||||
{
|
||||
auto column = columns.tryGetColumn(options, name);
|
||||
if (column && !isObject(column->type))
|
||||
{
|
||||
res.emplace_back(std::move(*column));
|
||||
continue;
|
||||
}
|
||||
|
||||
if (options.with_extended_objects)
|
||||
{
|
||||
auto object_column = object_columns.tryGetColumn(options, name);
|
||||
if (object_column)
|
||||
{
|
||||
res.emplace_back(std::move(*object_column));
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
if (options.with_virtuals)
|
||||
{
|
||||
auto it = virtual_columns.find(name);
|
||||
if (it != virtual_columns.end())
|
||||
{
|
||||
res.emplace_back(name, it->second);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no column {} in table", name);
|
||||
auto object_column = object_columns.tryGetColumn(options, column_name);
|
||||
if (object_column)
|
||||
return object_column;
|
||||
}
|
||||
|
||||
return res;
|
||||
if (options.with_virtuals)
|
||||
{
|
||||
auto it = virtual_columns.find(column_name);
|
||||
if (it != virtual_columns.end())
|
||||
return NameAndTypePair(column_name, it->second);
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
NameAndTypePair StorageSnapshot::getColumn(const GetColumnsOptions & options, const String & column_name) const
|
||||
{
|
||||
auto column = tryGetColumn(options, column_name);
|
||||
if (!column)
|
||||
throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no column {} in table", column_name);
|
||||
|
||||
return *column;
|
||||
}
|
||||
|
||||
Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) const
|
||||
|
@ -61,6 +61,10 @@ struct StorageSnapshot
|
||||
/// Get columns with types according to options only for requested names.
|
||||
NamesAndTypesList getColumnsByNames(const GetColumnsOptions & options, const Names & names) const;
|
||||
|
||||
/// Get column with type according to options for requested name.
|
||||
std::optional<NameAndTypePair> tryGetColumn(const GetColumnsOptions & options, const String & column_name) const;
|
||||
NameAndTypePair getColumn(const GetColumnsOptions & options, const String & column_name) const;
|
||||
|
||||
/// Block with ordinary + materialized + aliases + virtuals + subcolumns.
|
||||
Block getSampleBlockForColumns(const Names & column_names) const;
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
@ -346,39 +347,11 @@ namespace
|
||||
/* use_external_buffer */ false,
|
||||
/* skip_url_not_found_error */ skip_url_not_found_error);
|
||||
|
||||
ThreadGroupStatusPtr running_group = CurrentThread::isInitialized() && CurrentThread::get().getThreadGroup()
|
||||
? CurrentThread::get().getThreadGroup()
|
||||
: MainThreadStatus::getInstance().getThreadGroup();
|
||||
|
||||
ContextPtr query_context
|
||||
= CurrentThread::isInitialized() ? CurrentThread::get().getQueryContext() : nullptr;
|
||||
|
||||
auto worker_cleanup = [has_running_group = running_group == nullptr](ThreadStatus & thread_status)
|
||||
{
|
||||
if (has_running_group)
|
||||
thread_status.detachQuery(false);
|
||||
};
|
||||
|
||||
auto worker_setup = [query_context = std::move(query_context),
|
||||
running_group = std::move(running_group)](ThreadStatus & thread_status)
|
||||
{
|
||||
/// Save query context if any, because cache implementation needs it.
|
||||
if (query_context)
|
||||
thread_status.attachQueryContext(query_context);
|
||||
|
||||
/// To be able to pass ProfileEvents.
|
||||
if (running_group)
|
||||
thread_status.attachQuery(running_group);
|
||||
};
|
||||
|
||||
|
||||
return wrapReadBufferWithCompressionMethod(
|
||||
std::make_unique<ParallelReadBuffer>(
|
||||
std::move(read_buffer_factory),
|
||||
&IOThreadPool::get(),
|
||||
download_threads,
|
||||
std::move(worker_setup),
|
||||
std::move(worker_cleanup)),
|
||||
threadPoolCallbackRunner(IOThreadPool::get()),
|
||||
download_threads),
|
||||
chooseCompressionMethod(request_uri.getPath(), compression_method));
|
||||
}
|
||||
}
|
||||
|
@ -54,6 +54,7 @@ def get_packager_cmd(
|
||||
build_version: str,
|
||||
image_version: str,
|
||||
ccache_path: str,
|
||||
official: bool,
|
||||
) -> str:
|
||||
package_type = build_config["package_type"]
|
||||
comp = build_config["compiler"]
|
||||
@ -83,6 +84,9 @@ def get_packager_cmd(
|
||||
if _can_export_binaries(build_config):
|
||||
cmd += " --with-binaries=tests"
|
||||
|
||||
if official:
|
||||
cmd += " --official"
|
||||
|
||||
return cmd
|
||||
|
||||
|
||||
@ -254,9 +258,11 @@ def main():
|
||||
|
||||
logging.info("Got version from repo %s", version.string)
|
||||
|
||||
official_flag = pr_info.number == 0
|
||||
version_type = "testing"
|
||||
if "release" in pr_info.labels or "release-lts" in pr_info.labels:
|
||||
version_type = "stable"
|
||||
official_flag = True
|
||||
|
||||
update_version_local(REPO_COPY, version, version_type)
|
||||
|
||||
@ -290,7 +296,9 @@ def main():
|
||||
version.string,
|
||||
image_version,
|
||||
ccache_path,
|
||||
official=official_flag,
|
||||
)
|
||||
|
||||
logging.info("Going to run packager with %s", packager_cmd)
|
||||
|
||||
build_clickhouse_log = os.path.join(TEMP_PATH, "build_log")
|
||||
|
@ -233,7 +233,11 @@ if __name__ == "__main__":
|
||||
if ok_builds == 0 or some_builds_are_missing:
|
||||
summary_status = "error"
|
||||
|
||||
description = f"{ok_builds}/{total_builds} builds are OK"
|
||||
addition = ""
|
||||
if some_builds_are_missing:
|
||||
addition = "(some builds are missing)"
|
||||
|
||||
description = f"{ok_builds}/{total_builds} builds are OK {addition}"
|
||||
|
||||
print("::notice ::Report url: {}".format(url))
|
||||
|
||||
@ -244,3 +248,6 @@ if __name__ == "__main__":
|
||||
state=summary_status,
|
||||
target_url=url,
|
||||
)
|
||||
|
||||
if summary_status == "error":
|
||||
sys.exit(1)
|
||||
|
@ -20,8 +20,6 @@ class Description:
|
||||
|
||||
def __init__(self, pull_request):
|
||||
self.label_name = str()
|
||||
self.legal = False
|
||||
|
||||
self._parse(pull_request["bodyText"])
|
||||
|
||||
def _parse(self, text):
|
||||
@ -39,12 +37,6 @@ class Description:
|
||||
category = stripped
|
||||
next_category = False
|
||||
|
||||
if (
|
||||
stripped
|
||||
== "I hereby agree to the terms of the CLA available at: https://yandex.ru/legal/cla/?lang=en"
|
||||
):
|
||||
self.legal = True
|
||||
|
||||
category_headers = (
|
||||
"Category (leave one):",
|
||||
"Changelog category (leave one):",
|
||||
|
@ -10,13 +10,13 @@ from get_robot_token import get_parameter_from_ssm
|
||||
class ClickHouseHelper:
|
||||
def __init__(self, url=None):
|
||||
if url is None:
|
||||
self.url = get_parameter_from_ssm("clickhouse-test-stat-url2")
|
||||
self.auth = {
|
||||
"X-ClickHouse-User": get_parameter_from_ssm(
|
||||
"clickhouse-test-stat-login2"
|
||||
),
|
||||
"X-ClickHouse-Key": "",
|
||||
}
|
||||
url = get_parameter_from_ssm("clickhouse-test-stat-url")
|
||||
|
||||
self.url = url
|
||||
self.auth = {
|
||||
"X-ClickHouse-User": get_parameter_from_ssm("clickhouse-test-stat-login"),
|
||||
"X-ClickHouse-Key": get_parameter_from_ssm("clickhouse-test-stat-password"),
|
||||
}
|
||||
|
||||
@staticmethod
|
||||
def _insert_json_str_info_impl(url, auth, db, table, json_str):
|
||||
@ -179,7 +179,7 @@ def mark_flaky_tests(clickhouse_helper, check_name, test_results):
|
||||
check_name=check_name
|
||||
)
|
||||
|
||||
tests_data = clickhouse_helper.select_json_each_row("gh-data", query)
|
||||
tests_data = clickhouse_helper.select_json_each_row("default", query)
|
||||
master_failed_tests = {row["test_name"] for row in tests_data}
|
||||
logging.info("Found flaky tests: %s", ", ".join(master_failed_tests))
|
||||
|
||||
|
@ -59,3 +59,17 @@ def post_commit_status_to_file(file_path, description, state, report_url):
|
||||
with open(file_path, "w", encoding="utf-8") as f:
|
||||
out = csv.writer(f, delimiter="\t")
|
||||
out.writerow([state, report_url, description])
|
||||
|
||||
|
||||
def remove_labels(gh, pr_info, labels_names):
|
||||
repo = gh.get_repo(GITHUB_REPOSITORY)
|
||||
pull_request = repo.get_pull(pr_info.number)
|
||||
for label in labels_names:
|
||||
pull_request.remove_from_labels(label)
|
||||
|
||||
|
||||
def post_labels(gh, pr_info, labels_names):
|
||||
repo = gh.get_repo(GITHUB_REPOSITORY)
|
||||
pull_request = repo.get_pull(pr_info.number)
|
||||
for label in labels_names:
|
||||
pull_request.add_to_labels(label)
|
||||
|
@ -197,4 +197,8 @@ if __name__ == "__main__":
|
||||
report_url,
|
||||
CHECK_NAME,
|
||||
)
|
||||
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
|
||||
|
||||
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
|
||||
|
||||
if state == "error":
|
||||
sys.exit(1)
|
||||
|
@ -7,6 +7,7 @@ import platform
|
||||
import shutil
|
||||
import subprocess
|
||||
import time
|
||||
import sys
|
||||
from typing import Dict, List, Optional, Set, Tuple, Union
|
||||
|
||||
from github import Github
|
||||
@ -459,7 +460,10 @@ def main():
|
||||
NAME,
|
||||
)
|
||||
ch_helper = ClickHouseHelper()
|
||||
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
|
||||
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
|
||||
|
||||
if status == "error":
|
||||
sys.exit(1)
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
|
@ -234,7 +234,7 @@ def main():
|
||||
NAME,
|
||||
)
|
||||
ch_helper = ClickHouseHelper()
|
||||
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
|
||||
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
|
@ -114,4 +114,7 @@ if __name__ == "__main__":
|
||||
report_url,
|
||||
NAME,
|
||||
)
|
||||
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
|
||||
|
||||
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
|
||||
if status == "error":
|
||||
sys.exit(1)
|
||||
|
@ -204,11 +204,11 @@ if __name__ == "__main__":
|
||||
report_url,
|
||||
NAME,
|
||||
)
|
||||
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
|
||||
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
|
||||
|
||||
# Refuse other checks to run if fast test failed
|
||||
if state != "success":
|
||||
if "force-tests" in pr_info.labels:
|
||||
if "force-tests" in pr_info.labels and state != "error":
|
||||
print("'force-tests' enabled, will report success")
|
||||
else:
|
||||
sys.exit(1)
|
||||
|
@ -356,7 +356,7 @@ if __name__ == "__main__":
|
||||
report_url,
|
||||
check_name_with_group,
|
||||
)
|
||||
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
|
||||
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
|
||||
|
||||
if state != "success":
|
||||
if "force-tests" in pr_info.labels:
|
||||
|
@ -279,4 +279,8 @@ if __name__ == "__main__":
|
||||
report_url,
|
||||
check_name_with_group,
|
||||
)
|
||||
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
|
||||
|
||||
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
|
||||
|
||||
if state == "error":
|
||||
sys.exit(1)
|
||||
|
@ -271,5 +271,5 @@ if __name__ == "__main__":
|
||||
report_url,
|
||||
CHECK_NAME,
|
||||
)
|
||||
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
|
||||
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
|
||||
clear_autoscaling_group()
|
||||
|
@ -217,3 +217,6 @@ if __name__ == "__main__":
|
||||
post_commit_status(
|
||||
gh, pr_info.sha, check_name_with_group, message, status, report_url
|
||||
)
|
||||
|
||||
if status == "error":
|
||||
sys.exit(1)
|
||||
|
@ -236,6 +236,15 @@ class PRInfo:
|
||||
return True
|
||||
return False
|
||||
|
||||
def has_changes_in_submodules(self):
|
||||
if self.changed_files is None or not self.changed_files:
|
||||
return True
|
||||
|
||||
for f in self.changed_files:
|
||||
if "contrib" in f:
|
||||
return True
|
||||
return False
|
||||
|
||||
def can_skip_builds_and_use_version_from_master(self):
|
||||
# TODO: See a broken loop
|
||||
if "force tests" in self.labels:
|
||||
|
@ -8,7 +8,7 @@ from github import Github
|
||||
from env_helper import GITHUB_RUN_URL, GITHUB_REPOSITORY, GITHUB_SERVER_URL
|
||||
from pr_info import PRInfo
|
||||
from get_robot_token import get_best_robot_token
|
||||
from commit_status_helper import get_commit
|
||||
from commit_status_helper import get_commit, post_labels, remove_labels
|
||||
|
||||
NAME = "Run Check (actions)"
|
||||
|
||||
@ -22,6 +22,7 @@ OK_SKIP_LABELS = {"release", "pr-backport", "pr-cherrypick"}
|
||||
CAN_BE_TESTED_LABEL = "can be tested"
|
||||
DO_NOT_TEST_LABEL = "do not test"
|
||||
FORCE_TESTS_LABEL = "force tests"
|
||||
SUBMODULE_CHANGED_LABEL = "submodule changed"
|
||||
|
||||
# Individual trusted contirbutors who are not in any trusted organization.
|
||||
# Can be changed in runtime: we will append users that we learned to be in
|
||||
@ -81,6 +82,25 @@ TRUSTED_CONTRIBUTORS = {
|
||||
]
|
||||
}
|
||||
|
||||
MAP_CATEGORY_TO_LABEL = {
|
||||
"New Feature": "pr-feature",
|
||||
"Bug Fix": "pr-bugfix",
|
||||
"Bug Fix (user-visible misbehaviour in official stable or prestable release)": "pr-bugfix",
|
||||
"Improvement": "pr-improvement",
|
||||
"Performance Improvement": "pr-performance",
|
||||
"Backward Incompatible Change": "pr-backward-incompatible",
|
||||
"Build/Testing/Packaging Improvement": "pr-build",
|
||||
"Build Improvement": "pr-build",
|
||||
"Build/Testing Improvement": "pr-build",
|
||||
"Build": "pr-build",
|
||||
"Packaging Improvement": "pr-build",
|
||||
"Not for changelog (changelog entry is not required)": "pr-not-for-changelog",
|
||||
"Not for changelog": "pr-not-for-changelog",
|
||||
"Documentation (changelog entry is not required)": "pr-documentation",
|
||||
"Documentation": "pr-documentation",
|
||||
# 'Other': doesn't match anything
|
||||
}
|
||||
|
||||
|
||||
def pr_is_by_trusted_user(pr_user_login, pr_user_orgs):
|
||||
if pr_user_login.lower() in TRUSTED_CONTRIBUTORS:
|
||||
@ -168,7 +188,7 @@ def check_pr_description(pr_info):
|
||||
+ second_category
|
||||
+ "'"
|
||||
)
|
||||
return result_status[:140]
|
||||
return result_status[:140], category
|
||||
|
||||
elif re.match(
|
||||
r"(?i)^[>*_ ]*(short\s*description|change\s*log\s*entry)", lines[i]
|
||||
@ -190,30 +210,57 @@ def check_pr_description(pr_info):
|
||||
i += 1
|
||||
|
||||
if not category:
|
||||
return "Changelog category is empty"
|
||||
return "Changelog category is empty", category
|
||||
|
||||
# Filter out the PR categories that are not for changelog.
|
||||
if re.match(
|
||||
r"(?i)doc|((non|in|not|un)[-\s]*significant)|(not[ ]*for[ ]*changelog)",
|
||||
category,
|
||||
):
|
||||
return ""
|
||||
return "", category
|
||||
|
||||
if not entry:
|
||||
return f"Changelog entry required for category '{category}'"
|
||||
return f"Changelog entry required for category '{category}'", category
|
||||
|
||||
return ""
|
||||
return "", category
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
logging.basicConfig(level=logging.INFO)
|
||||
|
||||
pr_info = PRInfo(need_orgs=True, pr_event_from_api=True)
|
||||
pr_info = PRInfo(need_orgs=True, pr_event_from_api=True, need_changed_files=True)
|
||||
can_run, description, labels_state = should_run_checks_for_pr(pr_info)
|
||||
gh = Github(get_best_robot_token())
|
||||
commit = get_commit(gh, pr_info.sha)
|
||||
|
||||
description_report = check_pr_description(pr_info)[:139]
|
||||
description_report, category = check_pr_description(pr_info)
|
||||
pr_labels_to_add = []
|
||||
pr_labels_to_remove = []
|
||||
if (
|
||||
category in MAP_CATEGORY_TO_LABEL
|
||||
and MAP_CATEGORY_TO_LABEL[category] not in pr_info.labels
|
||||
):
|
||||
pr_labels_to_add.append(MAP_CATEGORY_TO_LABEL[category])
|
||||
|
||||
for label in pr_info.labels:
|
||||
if (
|
||||
label in MAP_CATEGORY_TO_LABEL.values()
|
||||
and category in MAP_CATEGORY_TO_LABEL
|
||||
and label != MAP_CATEGORY_TO_LABEL[category]
|
||||
):
|
||||
pr_labels_to_remove.append(label)
|
||||
|
||||
if pr_info.has_changes_in_submodules():
|
||||
pr_labels_to_add.append(SUBMODULE_CHANGED_LABEL)
|
||||
elif SUBMODULE_CHANGED_LABEL in pr_info.labels:
|
||||
pr_labels_to_remove.append(SUBMODULE_CHANGED_LABEL)
|
||||
|
||||
if pr_labels_to_add:
|
||||
post_labels(gh, pr_info, pr_labels_to_add)
|
||||
|
||||
if pr_labels_to_remove:
|
||||
remove_labels(gh, pr_info, pr_labels_to_remove)
|
||||
|
||||
if description_report:
|
||||
print("::notice ::Cannot run, description does not match the template")
|
||||
logging.info(
|
||||
@ -225,7 +272,7 @@ if __name__ == "__main__":
|
||||
)
|
||||
commit.create_status(
|
||||
context=NAME,
|
||||
description=description_report,
|
||||
description=description_report[:139],
|
||||
state="failure",
|
||||
target_url=url,
|
||||
)
|
||||
|
@ -147,4 +147,8 @@ if __name__ == "__main__":
|
||||
report_url,
|
||||
CHECK_NAME,
|
||||
)
|
||||
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
|
||||
|
||||
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
|
||||
|
||||
if state == "error":
|
||||
sys.exit(1)
|
||||
|
@ -176,4 +176,7 @@ if __name__ == "__main__":
|
||||
report_url,
|
||||
check_name,
|
||||
)
|
||||
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
|
||||
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
|
||||
|
||||
if state == "error":
|
||||
sys.exit(1)
|
||||
|
@ -117,4 +117,7 @@ if __name__ == "__main__":
|
||||
report_url,
|
||||
NAME,
|
||||
)
|
||||
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
|
||||
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
|
||||
|
||||
if state == "error":
|
||||
sys.exit(1)
|
||||
|
@ -173,4 +173,8 @@ if __name__ == "__main__":
|
||||
report_url,
|
||||
check_name,
|
||||
)
|
||||
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)
|
||||
|
||||
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
|
||||
|
||||
if state == "error":
|
||||
sys.exit(1)
|
||||
|
@ -52,6 +52,7 @@ MESSAGES_TO_RETRY = [
|
||||
"DB::Exception: New table appeared in database being dropped or detached. Try again",
|
||||
"is already started to be removing by another replica right now",
|
||||
"DB::Exception: Cannot enqueue query",
|
||||
"environment: line 1: wait_for: No record of process", # Something weird from bash internals, let's just retry
|
||||
"is executing longer than distributed_ddl_task_timeout" # FIXME
|
||||
]
|
||||
|
||||
@ -116,7 +117,7 @@ def clickhouse_execute_http(base_args, query, timeout=30, settings=None, default
|
||||
def clickhouse_execute(base_args, query, timeout=30, settings=None):
|
||||
return clickhouse_execute_http(base_args, query, timeout, settings).strip()
|
||||
|
||||
def clickhouse_execute_json(base_args, query, timeout=30, settings=None):
|
||||
def clickhouse_execute_json(base_args, query, timeout=60, settings=None):
|
||||
data = clickhouse_execute_http(base_args, query, timeout, settings, 'JSONEachRow')
|
||||
if not data:
|
||||
return None
|
||||
|
@ -280,4 +280,4 @@ def test_HDFS(start_cluster):
|
||||
|
||||
def test_schema_inference(start_cluster):
|
||||
error = node7.query_and_get_error("desc url('http://test.com`, 'TSVRaw'')")
|
||||
assert(error.find('ReadWriteBufferFromHTTPBase') == -1)
|
||||
assert error.find("ReadWriteBufferFromHTTPBase") == -1
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user