Merge branch 'master' of github.com:ClickHouse/ClickHouse into tighten-limits-functional-tests

This commit is contained in:
Alexey Milovidov 2024-08-12 23:56:06 +02:00
commit dea86ff20c
360 changed files with 2921 additions and 1910 deletions

View File

@ -187,14 +187,6 @@ else ()
set(NO_WHOLE_ARCHIVE --no-whole-archive)
endif ()
if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE")
# Can be lld or ld-lld or lld-13 or /path/to/lld.
if (LINKER_NAME MATCHES "lld")
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--gdb-index")
message (STATUS "Adding .gdb-index via --gdb-index linker option.")
endif ()
endif()
if (NOT (SANITIZE_COVERAGE OR WITH_COVERAGE)
AND (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE"
OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO"
@ -402,7 +394,7 @@ if ((NOT OS_LINUX AND NOT OS_ANDROID) OR (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")
set(ENABLE_GWP_ASAN OFF)
endif ()
option (ENABLE_FIU "Enable Fiu" ON)
option (ENABLE_LIBFIU "Enable libfiu" ON)
option(WERROR "Enable -Werror compiler option" ON)

View File

@ -179,7 +179,7 @@ else()
message(STATUS "Not using QPL")
endif ()
if (OS_LINUX AND ARCH_AMD64)
if (OS_LINUX AND ARCH_AMD64 AND NOT NO_SSE3_OR_HIGHER)
option (ENABLE_QATLIB "Enable Intel® QuickAssist Technology Library (QATlib)" ${ENABLE_LIBRARIES})
elseif(ENABLE_QATLIB)
message (${RECONFIGURE_MESSAGE_LEVEL} "QATLib is only supported on x86_64")

View File

@ -27,7 +27,7 @@ if (ENABLE_QAT_OUT_OF_TREE_BUILD)
${QAT_AL_INCLUDE_DIR}
${QAT_USDM_INCLUDE_DIR}
${ZSTD_LIBRARY_DIR})
target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_ZSTD_QAT_CODEC)
target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0)
add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin)
else () # In-tree build
message(STATUS "Intel QATZSTD in-tree build")
@ -78,7 +78,7 @@ else () # In-tree build
${QAT_USDM_INCLUDE_DIR}
${ZSTD_LIBRARY_DIR}
${LIBQAT_HEADER_DIR})
target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DENABLE_ZSTD_QAT_CODEC -DINTREE)
target_compile_definitions(_qatzstd_plugin PRIVATE -DDEBUGLEVEL=0 PUBLIC -DINTREE)
target_include_directories(_qatzstd_plugin SYSTEM PUBLIC $<BUILD_INTERFACE:${QATZSTD_SRC_DIR}> $<INSTALL_INTERFACE:include>)
add_library (ch_contrib::qatzstd_plugin ALIAS _qatzstd_plugin)
endif ()

View File

@ -1,20 +1,21 @@
if (NOT ENABLE_FIU)
message (STATUS "Not using fiu")
if (NOT ENABLE_LIBFIU)
message (STATUS "Not using libfiu")
return ()
endif ()
set(FIU_DIR "${ClickHouse_SOURCE_DIR}/contrib/libfiu/")
set(LIBFIU_DIR "${ClickHouse_SOURCE_DIR}/contrib/libfiu/")
set(FIU_SOURCES
${FIU_DIR}/libfiu/fiu.c
${FIU_DIR}/libfiu/fiu-rc.c
${FIU_DIR}/libfiu/backtrace.c
${FIU_DIR}/libfiu/wtable.c
set(LIBFIU_SOURCES
${LIBFIU_DIR}/libfiu/fiu.c
${LIBFIU_DIR}/libfiu/fiu-rc.c
${LIBFIU_DIR}/libfiu/backtrace.c
${LIBFIU_DIR}/libfiu/wtable.c
)
set(FIU_HEADERS "${FIU_DIR}/libfiu")
set(LIBFIU_HEADERS "${LIBFIU_DIR}/libfiu")
add_library(_fiu ${FIU_SOURCES})
target_compile_definitions(_fiu PUBLIC DUMMY_BACKTRACE)
target_include_directories(_fiu PUBLIC ${FIU_HEADERS})
add_library(ch_contrib::fiu ALIAS _fiu)
add_library(_libfiu ${LIBFIU_SOURCES})
target_compile_definitions(_libfiu PUBLIC DUMMY_BACKTRACE)
target_compile_definitions(_libfiu PUBLIC FIU_ENABLE)
target_include_directories(_libfiu PUBLIC ${LIBFIU_HEADERS})
add_library(ch_contrib::libfiu ALIAS _libfiu)

View File

@ -728,10 +728,6 @@ add_library(_qpl STATIC ${LIB_DEPS})
target_include_directories(_qpl
PUBLIC $<BUILD_INTERFACE:${QPL_PROJECT_DIR}/include/> $<INSTALL_INTERFACE:include>)
target_compile_definitions(_qpl
PUBLIC -DENABLE_QPL_COMPRESSION)
target_link_libraries(_qpl
PRIVATE ch_contrib::accel-config)

View File

@ -1,3 +1,5 @@
# docker build -t clickhouse/cctools .
# This is a hack to significantly reduce the build time of the clickhouse/binary-builder
# It's based on the assumption that we don't care of the cctools version so much
# It event does not depend on the clickhouse/fasttest in the `docker/images.json`
@ -30,5 +32,29 @@ RUN git clone https://github.com/tpoechtrager/cctools-port.git \
&& cd ../.. \
&& rm -rf cctools-port
#
# GDB
#
# ld from binutils is 2.38, which has the following error:
#
# DWARF error: invalid or unhandled FORM value: 0x23
#
ENV LD=ld.lld-${LLVM_VERSION}
ARG GDB_VERSION=15.1
RUN apt-get update \
&& apt-get install --yes \
libgmp-dev \
libmpfr-dev \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
RUN wget https://sourceware.org/pub/gdb/releases/gdb-$GDB_VERSION.tar.gz \
&& tar -xvf gdb-$GDB_VERSION.tar.gz \
&& cd gdb-$GDB_VERSION \
&& ./configure --prefix=/opt/gdb \
&& make -j $(nproc) \
&& make install \
&& rm -fr gdb-$GDB_VERSION gdb-$GDB_VERSION.tar.gz
FROM scratch
COPY --from=builder /cctools /cctools
COPY --from=builder /opt/gdb /opt/gdb

View File

@ -83,7 +83,7 @@ RUN arch=${TARGETARCH:-amd64} \
# Give suid to gdb to grant it attach permissions
# chmod 777 to make the container user independent
RUN chmod u+s /usr/bin/gdb \
RUN chmod u+s /opt/gdb/bin/gdb \
&& mkdir -p /var/lib/clickhouse \
&& chmod 777 /var/lib/clickhouse

View File

@ -11,7 +11,6 @@ RUN apt-get update \
curl \
default-jre \
g++ \
gdb \
iproute2 \
krb5-user \
libicu-dev \
@ -73,3 +72,6 @@ maxClientCnxns=80' > /opt/zookeeper/conf/zoo.cfg && \
ENV TZ=Etc/UTC
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
COPY --from=clickhouse/cctools:0d6b90a7a490 /opt/gdb /opt/gdb
ENV PATH="/opt/gdb/bin:${PATH}"

View File

@ -30,7 +30,6 @@ RUN apt-get update \
luajit \
libssl-dev \
libcurl4-openssl-dev \
gdb \
default-jdk \
software-properties-common \
libkrb5-dev \
@ -87,6 +86,8 @@ COPY modprobe.sh /usr/local/bin/modprobe
COPY dockerd-entrypoint.sh /usr/local/bin/
COPY misc/ /misc/
COPY --from=clickhouse/cctools:0d6b90a7a490 /opt/gdb /opt/gdb
ENV PATH="/opt/gdb/bin:${PATH}"
# Same options as in test/base/Dockerfile
# (in case you need to override them in tests)

View File

@ -9,7 +9,6 @@ RUN apt-get update \
curl \
dmidecode \
g++ \
gdb \
git \
gnuplot \
imagemagick \
@ -42,6 +41,9 @@ RUN pip3 --no-cache-dir install -r requirements.txt
COPY run.sh /
COPY --from=clickhouse/cctools:0d6b90a7a490 /opt/gdb /opt/gdb
ENV PATH="/opt/gdb/bin:${PATH}"
CMD ["bash", "/run.sh"]
# docker run --network=host --volume <workspace>:/workspace --volume=<output>:/output -e PR_TO_TEST=<> -e SHA_TO_TEST=<> clickhouse/performance-comparison

View File

@ -54,8 +54,6 @@ source /utils.lib
/usr/share/clickhouse-test/config/install.sh
./setup_minio.sh stateless
./mc admin trace clickminio > /test_output/minio.log &
MC_ADMIN_PID=$!
./setup_hdfs_minicluster.sh
@ -176,6 +174,55 @@ done
setup_logs_replication
attach_gdb_to_clickhouse
# create tables for minio log webhooks
clickhouse-client --query "CREATE TABLE minio_audit_logs
(
log String,
event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(trim(BOTH '\"' FROM JSONExtractRaw(log, 'time')), 9, 'UTC')
)
ENGINE = MergeTree
ORDER BY tuple()"
clickhouse-client --query "CREATE TABLE minio_server_logs
(
log String,
event_time DateTime64(9) MATERIALIZED parseDateTime64BestEffortOrZero(trim(BOTH '\"' FROM JSONExtractRaw(log, 'time')), 9, 'UTC')
)
ENGINE = MergeTree
ORDER BY tuple()"
# create minio log webhooks for both audit and server logs
# use async inserts to avoid creating too many parts
./mc admin config set clickminio logger_webhook:ch_server_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_server_logs%20FORMAT%20LineAsString" queue_size=1000000 batch_size=500
./mc admin config set clickminio audit_webhook:ch_audit_webhook endpoint="http://localhost:8123/?async_insert=1&wait_for_async_insert=0&async_insert_busy_timeout_min_ms=5000&async_insert_busy_timeout_max_ms=5000&async_insert_max_query_number=1000&async_insert_max_data_size=10485760&query=INSERT%20INTO%20minio_audit_logs%20FORMAT%20LineAsString" queue_size=1000000 batch_size=500
max_retries=100
retry=1
while [ $retry -le $max_retries ]; do
echo "clickminio restart attempt $retry:"
output=$(./mc admin service restart clickminio --wait --json 2>&1 | jq -r .status)
echo "Output of restart status: $output"
expected_output="success
success"
if [ "$output" = "$expected_output" ]; then
echo "Restarted clickminio successfully."
break
fi
sleep 1
retry=$((retry + 1))
done
if [ $retry -gt $max_retries ]; then
echo "Failed to restart clickminio after $max_retries attempts."
fi
./mc admin trace clickminio > /test_output/minio.log &
MC_ADMIN_PID=$!
function fn_exists() {
declare -F "$1" > /dev/null;
}
@ -339,6 +386,14 @@ do
fi
done
# collect minio audit and server logs
# wait for minio to flush its batch if it has any
sleep 1
clickhouse-client -q "SYSTEM FLUSH ASYNC INSERT QUEUE"
clickhouse-client -q "SELECT log FROM minio_audit_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_audit_logs.jsonl.zst' FORMAT JSONEachRow"
clickhouse-client -q "SELECT log FROM minio_server_logs ORDER BY event_time INTO OUTFILE '/test_output/minio_server_logs.jsonl.zst' FORMAT JSONEachRow"
# Stop server so we can safely read data with clickhouse-local.
# Why do we read data with clickhouse-local?
# Because it's the simplest way to read it when server has crashed.

View File

@ -44,7 +44,6 @@ RUN apt-get update \
bash \
bsdmainutils \
build-essential \
gdb \
git \
gperf \
moreutils \
@ -58,3 +57,6 @@ RUN apt-get update \
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
COPY process_functional_tests_result.py /
COPY --from=clickhouse/cctools:0d6b90a7a490 /opt/gdb /opt/gdb
ENV PATH="/opt/gdb/bin:${PATH}"

View File

@ -14,7 +14,7 @@ Each functional test sends one or multiple queries to the running ClickHouse ser
Tests are located in `queries` directory. There are two subdirectories: `stateless` and `stateful`. Stateless tests run queries without any preloaded test data - they often create small synthetic datasets on the fly, within the test itself. Stateful tests require preloaded test data from ClickHouse and it is available to general public.
Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`.
Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`.
:::note
A common mistake when testing data types `DateTime` and `DateTime64` is assuming that the server uses a specific time zone (e.g. "UTC"). This is not the case, time zones in CI test runs
@ -38,7 +38,7 @@ For more options, see `tests/clickhouse-test --help`. You can simply run all tes
### Adding a New Test
To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client --multiquery < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`.
To add new test, create a `.sql` or `.sh` file in `queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`.
Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables.

View File

@ -24,6 +24,7 @@ Columns:
- `num_rebalance_revocations`, (UInt64) - number of times the consumer was revoked its partitions
- `num_rebalance_assignments`, (UInt64) - number of times the consumer was assigned to Kafka cluster
- `is_currently_used`, (UInt8) - consumer is in use
- `last_used`, (UInt64) - last time this consumer was in use, unix time in microseconds
- `rdkafka_stat` (String) - library internal statistic. See https://github.com/ClickHouse/librdkafka/blob/master/STATISTICS.md . Set `statistics_interval_ms` to 0 disable, default is 3000 (once in three seconds).
Example:

View File

@ -8,26 +8,28 @@ sidebar_label: STATISTICS
The following operations are available:
- `ALTER TABLE [db].table ADD STATISTICS (columns list) TYPE (type list)` - Adds statistic description to tables metadata.
- `ALTER TABLE [db].table ADD STATISTICS [IF NOT EXISTS] (column list) TYPE (type list)` - Adds statistic description to tables metadata.
- `ALTER TABLE [db].table MODIFY STATISTICS (columns list) TYPE (type list)` - Modifies statistic description to tables metadata.
- `ALTER TABLE [db].table MODIFY STATISTICS (column list) TYPE (type list)` - Modifies statistic description to tables metadata.
- `ALTER TABLE [db].table DROP STATISTICS (columns list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns.
- `ALTER TABLE [db].table DROP STATISTICS [IF EXISTS] (column list)` - Removes statistics from the metadata of the specified columns and deletes all statistics objects in all parts for the specified columns.
- `ALTER TABLE [db].table CLEAR STATISTICS (columns list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`.
- `ALTER TABLE [db].table CLEAR STATISTICS [IF EXISTS] (column list)` - Deletes all statistics objects in all parts for the specified columns. Statistics objects can be rebuild using `ALTER TABLE MATERIALIZE STATISTICS`.
- `ALTER TABLE [db.]table MATERIALIZE STATISTICS (columns list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations).
- `ALTER TABLE [db.]table MATERIALIZE STATISTICS [IF EXISTS] (column list)` - Rebuilds the statistic for columns. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations).
The first two commands are lightweight in a sense that they only change metadata or remove files.
Also, they are replicated, syncing statistics metadata via ZooKeeper.
There is an example adding two statistics types to two columns:
## Example:
Adding two statistics types to two columns:
```
ALTER TABLE t1 MODIFY STATISTICS c, d TYPE TDigest, Uniq;
```
:::note
Statistic manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants).
Statistic are supported only for [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine tables (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants).
:::

View File

@ -23,30 +23,30 @@ slug: /zh/operations/external-authenticators/kerberos
示例 (进入 `config.xml`):
```xml
<yandex>
<clickhouse>
<!- ... -->
<kerberos />
</yandex>
</clickhouse>
```
主体规范:
```xml
<yandex>
<clickhouse>
<!- ... -->
<kerberos>
<principal>HTTP/clickhouse.example.com@EXAMPLE.COM</principal>
</kerberos>
</yandex>
</clickhouse>
```
按领域过滤:
```xml
<yandex>
<clickhouse>
<!- ... -->
<kerberos>
<realm>EXAMPLE.COM</realm>
</kerberos>
</yandex>
</clickhouse>
```
!!! warning "注意"
@ -74,7 +74,7 @@ Kerberos主体名称格式通常遵循以下模式:
示例 (进入 `users.xml`):
```
<yandex>
<clickhouse>
<!- ... -->
<users>
<!- ... -->
@ -85,7 +85,7 @@ Kerberos主体名称格式通常遵循以下模式:
</kerberos>
</my_user>
</users>
</yandex>
</clickhouse>
```
!!! warning "警告"

View File

@ -223,7 +223,7 @@ std::vector<String> Client::loadWarningMessages()
size_t rows = packet.block.rows();
for (size_t i = 0; i < rows; ++i)
messages.emplace_back(column[i].get<String>());
messages.emplace_back(column[i].safeGet<String>());
}
continue;

View File

@ -95,7 +95,7 @@ void SetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co
client->zookeeper->set(
client->getAbsolutePath(query->args[0].safeGet<String>()),
query->args[1].safeGet<String>(),
static_cast<Int32>(query->args[2].get<Int32>()));
static_cast<Int32>(query->args[2].safeGet<Int32>()));
}
bool CreateCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, Expected & expected) const
@ -494,7 +494,7 @@ void RMCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con
{
Int32 version{-1};
if (query->args.size() == 2)
version = static_cast<Int32>(query->args[1].get<Int32>());
version = static_cast<Int32>(query->args[1].safeGet<Int32>());
client->zookeeper->remove(client->getAbsolutePath(query->args[0].safeGet<String>()), version);
}
@ -549,7 +549,7 @@ void ReconfigCommand::execute(const DB::ASTKeeperQuery * query, DB::KeeperClient
String leaving;
String new_members;
auto operation = query->args[0].get<ReconfigCommand::Operation>();
auto operation = query->args[0].safeGet<ReconfigCommand::Operation>();
switch (operation)
{
case static_cast<UInt8>(ReconfigCommand::Operation::ADD):

View File

@ -0,0 +1 @@
../../../tests/config/config.d/transactions.xml

View File

@ -780,12 +780,12 @@ AggregateFunctionPtr createAggregateFunctionGroupArray(
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
if ((type == Field::Types::Int64 && parameters[0].get<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[0].get<UInt64>() == 0))
if ((type == Field::Types::Int64 && parameters[0].safeGet<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[0].safeGet<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
has_limit = true;
max_elems = parameters[0].get<UInt64>();
max_elems = parameters[0].safeGet<UInt64>();
}
else
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
@ -816,11 +816,11 @@ AggregateFunctionPtr createAggregateFunctionGroupArraySample(
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
if ((type == Field::Types::Int64 && parameters[i].get<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[i].get<UInt64>() == 0))
if ((type == Field::Types::Int64 && parameters[i].safeGet<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[i].safeGet<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
return parameters[i].get<UInt64>();
return parameters[i].safeGet<UInt64>();
};
UInt64 max_elems = get_parameter(0);

View File

@ -83,16 +83,16 @@ public:
if (version == 1)
{
for (size_t i = 0; i < arr_size; ++i)
set.insert(static_cast<T>((*data_column)[offset + i].get<T>()));
set.insert(static_cast<T>((*data_column)[offset + i].safeGet<T>()));
}
else if (!set.empty())
{
typename State::Set new_set;
for (size_t i = 0; i < arr_size; ++i)
{
typename State::Set::LookupResult set_value = set.find(static_cast<T>((*data_column)[offset + i].get<T>()));
typename State::Set::LookupResult set_value = set.find(static_cast<T>((*data_column)[offset + i].safeGet<T>()));
if (set_value != nullptr)
new_set.insert(static_cast<T>((*data_column)[offset + i].get<T>()));
new_set.insert(static_cast<T>((*data_column)[offset + i].safeGet<T>()));
}
set = std::move(new_set);
}

View File

@ -269,12 +269,12 @@ AggregateFunctionPtr createAggregateFunctionMoving(
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name);
if ((type == Field::Types::Int64 && parameters[0].get<Int64>() <= 0) ||
(type == Field::Types::UInt64 && parameters[0].get<UInt64>() == 0))
if ((type == Field::Types::Int64 && parameters[0].safeGet<Int64>() <= 0) ||
(type == Field::Types::UInt64 && parameters[0].safeGet<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name);
limit_size = true;
max_elems = parameters[0].get<UInt64>();
max_elems = parameters[0].safeGet<UInt64>();
}
else
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,

View File

@ -397,11 +397,11 @@ AggregateFunctionPtr createAggregateFunctionGroupArray(
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
if ((type == Field::Types::Int64 && parameters[0].get<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[0].get<UInt64>() == 0))
if ((type == Field::Types::Int64 && parameters[0].safeGet<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[0].safeGet<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
max_elems = parameters[0].get<UInt64>();
max_elems = parameters[0].safeGet<UInt64>();
}
else
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,

View File

@ -247,7 +247,7 @@ AggregateFunctionPtr createAggregateFunctionGroupConcat(
if (type != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First parameter for aggregate function {} should be string", name);
delimiter = parameters[0].get<String>();
delimiter = parameters[0].safeGet<String>();
}
if (parameters.size() == 2)
{
@ -256,12 +256,12 @@ AggregateFunctionPtr createAggregateFunctionGroupConcat(
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number", name);
if ((type == Field::Types::Int64 && parameters[1].get<Int64>() <= 0) ||
(type == Field::Types::UInt64 && parameters[1].get<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number, got: {}", name, parameters[1].get<Int64>());
if ((type == Field::Types::Int64 && parameters[1].safeGet<Int64>() <= 0) ||
(type == Field::Types::UInt64 && parameters[1].safeGet<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number, got: {}", name, parameters[1].safeGet<Int64>());
has_limit = true;
limit = parameters[1].get<UInt64>();
limit = parameters[1].safeGet<UInt64>();
}
if (has_limit)

View File

@ -323,12 +323,12 @@ AggregateFunctionPtr createAggregateFunctionGroupUniqArray(
if (type != Field::Types::Int64 && type != Field::Types::UInt64)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
if ((type == Field::Types::Int64 && parameters[0].get<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[0].get<UInt64>() == 0))
if ((type == Field::Types::Int64 && parameters[0].safeGet<Int64>() < 0) ||
(type == Field::Types::UInt64 && parameters[0].safeGet<UInt64>() == 0))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive number", name);
limit_size = true;
max_elems = parameters[0].get<UInt64>();
max_elems = parameters[0].safeGet<UInt64>();
}
else
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,

View File

@ -238,7 +238,7 @@ public:
if (params[0].getType() != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a String", getName());
const auto & param = params[0].get<String>();
const auto & param = params[0].safeGet<String>();
if (param == "two-sided")
alternative = Alternative::TwoSided;
else if (param == "less")
@ -255,7 +255,7 @@ public:
if (params[1].getType() != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require second parameter to be a String", getName());
method = params[1].get<String>();
method = params[1].safeGet<String>();
if (method != "auto" && method != "exact" && method != "asymp" && method != "asymptotic")
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown method in aggregate function {}. "
"It must be one of: 'auto', 'exact', 'asymp' (or 'asymptotic')", getName());

View File

@ -181,7 +181,7 @@ public:
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a UInt64", getName());
total_buckets = params[0].get<UInt64>();
total_buckets = params[0].safeGet<UInt64>();
this->x_type = WhichDataType(arguments[0]).idx;
this->y_type = WhichDataType(arguments[1]).idx;

View File

@ -152,7 +152,7 @@ public:
if (params[0].getType() != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a String", getName());
const auto & param = params[0].get<String>();
const auto & param = params[0].safeGet<String>();
if (param == "two-sided")
alternative = Alternative::TwoSided;
else if (param == "less")
@ -169,7 +169,7 @@ public:
if (params[1].getType() != Field::Types::UInt64)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require second parameter to be a UInt64", getName());
continuity_correction = static_cast<bool>(params[1].get<UInt64>());
continuity_correction = static_cast<bool>(params[1].safeGet<UInt64>());
}
String getName() const override

View File

@ -117,7 +117,7 @@ public:
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} requires relative accuracy parameter with Float64 type", getName());
relative_accuracy = relative_accuracy_field.get<Float64>();
relative_accuracy = relative_accuracy_field.safeGet<Float64>();
if (relative_accuracy <= 0 || relative_accuracy >= 1 || isNaN(relative_accuracy))
throw Exception(
@ -147,9 +147,9 @@ public:
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} requires accuracy parameter with integer type", getName());
if (accuracy_field.getType() == Field::Types::Int64)
accuracy = accuracy_field.get<Int64>();
accuracy = accuracy_field.safeGet<Int64>();
else
accuracy = accuracy_field.get<UInt64>();
accuracy = accuracy_field.safeGet<UInt64>();
if (accuracy <= 0)
throw Exception(

View File

@ -300,12 +300,12 @@ public:
/// Compatibility with previous versions.
if (value.getType() == Field::Types::Decimal32)
{
auto source = value.get<DecimalField<Decimal32>>();
auto source = value.safeGet<DecimalField<Decimal32>>();
value = DecimalField<Decimal128>(source.getValue(), source.getScale());
}
else if (value.getType() == Field::Types::Decimal64)
{
auto source = value.get<DecimalField<Decimal64>>();
auto source = value.safeGet<DecimalField<Decimal64>>();
value = DecimalField<Decimal128>(source.getValue(), source.getScale());
}
@ -355,7 +355,7 @@ public:
/// Compatibility with previous versions.
if (value.getType() == Field::Types::Decimal128)
{
auto source = value.get<DecimalField<Decimal128>>();
auto source = value.safeGet<DecimalField<Decimal128>>();
WhichDataType value_type(values_types[col_idx]);
if (value_type.isDecimal32())
{
@ -560,7 +560,7 @@ private:
template <typename FieldType>
bool compareImpl(FieldType & x) const
{
auto val = rhs.get<FieldType>();
auto val = rhs.safeGet<FieldType>();
if (val > x)
{
x = val;
@ -600,7 +600,7 @@ private:
template <typename FieldType>
bool compareImpl(FieldType & x) const
{
auto val = rhs.get<FieldType>();
auto val = rhs.safeGet<FieldType>();
if (val < x)
{
x = val;

View File

@ -137,7 +137,7 @@ private:
if (constant_node_value.getType() != Field::Types::Which::Tuple)
return {};
const auto & constant_tuple = constant_node_value.get<const Tuple &>();
const auto & constant_tuple = constant_node_value.safeGet<const Tuple &>();
const auto & function_arguments_nodes = function_node_typed.getArguments().getNodes();
size_t function_arguments_nodes_size = function_arguments_nodes.size();

View File

@ -89,7 +89,7 @@ public:
if (!pattern || !isString(pattern->getResultType()))
continue;
auto regexp = likePatternToRegexp(pattern->getValue().get<String>());
auto regexp = likePatternToRegexp(pattern->getValue().safeGet<String>());
/// Case insensitive. Works with UTF-8 as well.
if (is_ilike)
regexp = "(?i)" + regexp;

View File

@ -68,10 +68,10 @@ void optimizeFunctionEmpty(QueryTreeNodePtr &, FunctionNode & function_node, Col
String getSubcolumnNameForElement(const Field & value, const DataTypeTuple & data_type_tuple)
{
if (value.getType() == Field::Types::String)
return value.get<const String &>();
return value.safeGet<const String &>();
if (value.getType() == Field::Types::UInt64)
return data_type_tuple.getNameByPosition(value.get<UInt64>());
return data_type_tuple.getNameByPosition(value.safeGet<UInt64>());
return "";
}
@ -79,7 +79,7 @@ String getSubcolumnNameForElement(const Field & value, const DataTypeTuple & dat
String getSubcolumnNameForElement(const Field & value, const DataTypeVariant &)
{
if (value.getType() == Field::Types::String)
return value.get<const String &>();
return value.safeGet<const String &>();
return "";
}

View File

@ -187,7 +187,7 @@ FunctionNodePtr createFusedQuantilesNode(std::vector<QueryTreeNodePtr *> & nodes
/// Sort nodes and parameters in ascending order of quantile level
std::vector<size_t> permutation(nodes.size());
iota(permutation.data(), permutation.size(), size_t(0));
std::sort(permutation.begin(), permutation.end(), [&](size_t i, size_t j) { return parameters[i].get<Float64>() < parameters[j].get<Float64>(); });
std::sort(permutation.begin(), permutation.end(), [&](size_t i, size_t j) { return parameters[i].safeGet<Float64>() < parameters[j].safeGet<Float64>(); });
std::vector<QueryTreeNodePtr *> new_nodes;
new_nodes.reserve(permutation.size());

View File

@ -134,8 +134,8 @@ public:
return;
std::set<std::string> string_values;
string_values.insert(first_literal->getValue().get<std::string>());
string_values.insert(second_literal->getValue().get<std::string>());
string_values.insert(first_literal->getValue().safeGet<std::string>());
string_values.insert(second_literal->getValue().safeGet<std::string>());
changeIfArguments(*function_if_node, string_values, context);
wrapIntoToString(*function_node, std::move(modified_if_node), context);
@ -163,7 +163,7 @@ public:
if (!isArray(literal_to->getResultType()) || !isString(literal_default->getResultType()))
return;
auto array_to = literal_to->getValue().get<Array>();
auto array_to = literal_to->getValue().safeGet<Array>();
if (array_to.empty())
return;
@ -178,9 +178,9 @@ public:
std::set<std::string> string_values;
for (const auto & value : array_to)
string_values.insert(value.get<std::string>());
string_values.insert(value.safeGet<std::string>());
string_values.insert(literal_default->getValue().get<std::string>());
string_values.insert(literal_default->getValue().safeGet<std::string>());
changeTransformArguments(*function_modified_transform_node, string_values, context);
wrapIntoToString(*function_node, std::move(modified_transform_node), context);

View File

@ -54,7 +54,7 @@ public:
}
else if (function_node->getFunctionName() == "sum" &&
first_argument_constant_literal.getType() == Field::Types::UInt64 &&
first_argument_constant_literal.get<UInt64>() == 1)
first_argument_constant_literal.safeGet<UInt64>() == 1)
{
function_node->getArguments().getNodes().clear();
resolveAggregateFunctionNodeByName(*function_node, "count");

View File

@ -143,13 +143,13 @@ private:
const auto & column_type = column_node_typed.getColumnType().get();
if (isDateOrDate32(column_type))
{
start_date_or_date_time = date_lut.dateToString(range.first.get<DateLUTImpl::Time>());
end_date_or_date_time = date_lut.dateToString(range.second.get<DateLUTImpl::Time>());
start_date_or_date_time = date_lut.dateToString(range.first.safeGet<DateLUTImpl::Time>());
end_date_or_date_time = date_lut.dateToString(range.second.safeGet<DateLUTImpl::Time>());
}
else if (isDateTime(column_type) || isDateTime64(column_type))
{
start_date_or_date_time = date_lut.timeToString(range.first.get<DateLUTImpl::Time>());
end_date_or_date_time = date_lut.timeToString(range.second.get<DateLUTImpl::Time>());
start_date_or_date_time = date_lut.timeToString(range.first.safeGet<DateLUTImpl::Time>());
end_date_or_date_time = date_lut.timeToString(range.second.safeGet<DateLUTImpl::Time>());
}
else [[unlikely]]
return {};

View File

@ -60,7 +60,7 @@ public:
{
const auto & second_const_value = second_const_node->getValue();
if (second_const_value.isNull()
|| (lower_name == "sum" && isInt64OrUInt64FieldType(second_const_value.getType()) && second_const_value.get<UInt64>() == 0
|| (lower_name == "sum" && isInt64OrUInt64FieldType(second_const_value.getType()) && second_const_value.safeGet<UInt64>() == 0
&& !if_node->getResultType()->isNullable()))
{
/// avg(if(cond, a, null)) -> avgIf(a::ResultTypeIf, cond)
@ -89,7 +89,7 @@ public:
{
const auto & first_const_value = first_const_node->getValue();
if (first_const_value.isNull()
|| (lower_name == "sum" && isInt64OrUInt64FieldType(first_const_value.getType()) && first_const_value.get<UInt64>() == 0
|| (lower_name == "sum" && isInt64OrUInt64FieldType(first_const_value.getType()) && first_const_value.safeGet<UInt64>() == 0
&& !if_node->getResultType()->isNullable()))
{
/// avg(if(cond, null, a) -> avgIf(a::ResultTypeIf, !cond))

View File

@ -66,7 +66,7 @@ public:
resolveAggregateFunctionNodeByName(*function_node, "countIf");
if (constant_value_literal.get<UInt64>() != 1)
if (constant_value_literal.safeGet<UInt64>() != 1)
{
/// Rewrite `sumIf(123, cond)` into `123 * countIf(cond)`
node = getMultiplyFunction(std::move(multiplier_node), node);
@ -105,8 +105,8 @@ public:
const auto & if_true_condition_constant_value_literal = if_true_condition_constant_node->getValue();
const auto & if_false_condition_constant_value_literal = if_false_condition_constant_node->getValue();
auto if_true_condition_value = if_true_condition_constant_value_literal.get<UInt64>();
auto if_false_condition_value = if_false_condition_constant_value_literal.get<UInt64>();
auto if_true_condition_value = if_true_condition_constant_value_literal.safeGet<UInt64>();
auto if_false_condition_value = if_false_condition_constant_value_literal.safeGet<UInt64>();
if (if_false_condition_value == 0)
{

View File

@ -471,7 +471,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_express
std::shared_ptr<Collator> collator;
if (order_by_element.getCollation())
collator = std::make_shared<Collator>(order_by_element.getCollation()->as<ASTLiteral &>().value.get<String &>());
collator = std::make_shared<Collator>(order_by_element.getCollation()->as<ASTLiteral &>().value.safeGet<String &>());
const auto & sort_expression_ast = order_by_element.children.at(0);
auto sort_expression = buildExpression(sort_expression_ast, context);

View File

@ -1273,7 +1273,7 @@ QueryTreeNodePtr IdentifierResolver::matchArrayJoinSubcolumns(
const auto & constant_node_value = constant_node.getValue();
if (constant_node_value.getType() == Field::Types::String)
{
array_join_subcolumn_prefix = constant_node_value.get<String>() + ".";
array_join_subcolumn_prefix = constant_node_value.safeGet<String>() + ".";
array_join_parent_column = argument_nodes.at(0).get();
}
}
@ -1287,7 +1287,7 @@ QueryTreeNodePtr IdentifierResolver::matchArrayJoinSubcolumns(
if (!second_argument || second_argument->getValue().getType() != Field::Types::String)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected constant string as second argument of getSubcolumn function {}", resolved_function->dumpTree());
const auto & resolved_subcolumn_path = second_argument->getValue().get<String &>();
const auto & resolved_subcolumn_path = second_argument->getValue().safeGet<String &>();
if (!startsWith(resolved_subcolumn_path, array_join_subcolumn_prefix))
return {};
@ -1331,7 +1331,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveExpressionFromArrayJoinExpression
size_t nested_function_arguments_size = nested_function_arguments.size();
const auto & nested_keys_names_constant_node = nested_function_arguments[0]->as<ConstantNode & >();
const auto & nested_keys_names = nested_keys_names_constant_node.getValue().get<Array &>();
const auto & nested_keys_names = nested_keys_names_constant_node.getValue().safeGet<Array &>();
size_t nested_keys_names_size = nested_keys_names.size();
if (nested_keys_names_size == nested_function_arguments_size - 1)
@ -1344,7 +1344,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveExpressionFromArrayJoinExpression
auto array_join_column = std::make_shared<ColumnNode>(array_join_column_expression_typed.getColumn(),
array_join_column_expression_typed.getColumnSource());
const auto & nested_key_name = nested_keys_names[i - 1].get<String &>();
const auto & nested_key_name = nested_keys_names[i - 1].safeGet<String &>();
Identifier nested_identifier = Identifier(nested_key_name);
array_join_resolved_expression = wrapExpressionNodeInTupleElement(array_join_column, nested_identifier, scope.context);
break;

View File

@ -748,11 +748,11 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_
UInt64 pos;
if (constant_node->getValue().getType() == Field::Types::UInt64)
{
pos = constant_node->getValue().get<UInt64>();
pos = constant_node->getValue().safeGet<UInt64>();
}
else // Int64
{
auto value = constant_node->getValue().get<Int64>();
auto value = constant_node->getValue().safeGet<Int64>();
if (value > 0)
pos = value;
else

View File

@ -99,7 +99,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes&
"Invalid type in set. Expected tuple, got {}",
value.getTypeName());
const auto & tuple = value.template get<const Tuple &>();
const auto & tuple = value.template safeGet<const Tuple &>();
const DataTypePtr & value_type = value_types[collection_index];
const DataTypes & tuple_value_type = typeid_cast<const DataTypeTuple *>(value_type.get())->getElements();
@ -175,15 +175,15 @@ Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const
if (rhs_which_type.isArray())
{
const DataTypeArray * value_array_type = assert_cast<const DataTypeArray *>(value_type.get());
size_t value_array_size = value.get<const Array &>().size();
size_t value_array_size = value.safeGet<const Array &>().size();
DataTypes value_types(value_array_size, value_array_type->getNestedType());
result_block = createBlockFromCollection(value.get<const Array &>(), value_types, set_element_types, transform_null_in);
result_block = createBlockFromCollection(value.safeGet<const Array &>(), value_types, set_element_types, transform_null_in);
}
else if (rhs_which_type.isTuple())
{
const DataTypeTuple * value_tuple_type = assert_cast<const DataTypeTuple *>(value_type.get());
const DataTypes & value_types = value_tuple_type->getElements();
result_block = createBlockFromCollection(value.get<const Tuple &>(), value_types, set_element_types, transform_null_in);
result_block = createBlockFromCollection(value.safeGet<const Tuple &>(), value_types, set_element_types, transform_null_in);
}
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,

View File

@ -126,7 +126,7 @@ std::vector<Strings> BackupSettings::Util::clusterHostIDsFromAST(const IAST & as
throw Exception(
ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS,
"Setting cluster_host_ids has wrong format, must be array of arrays of string literals");
const auto & replicas = array_of_replicas->value.get<const Array &>();
const auto & replicas = array_of_replicas->value.safeGet<const Array &>();
res[i].resize(replicas.size());
for (size_t j = 0; j != replicas.size(); ++j)
{
@ -135,7 +135,7 @@ std::vector<Strings> BackupSettings::Util::clusterHostIDsFromAST(const IAST & as
throw Exception(
ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS,
"Setting cluster_host_ids has wrong format, must be array of arrays of string literals");
res[i][j] = replica.get<const String &>();
res[i][j] = replica.safeGet<const String &>();
}
}
}

View File

@ -490,6 +490,8 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context
/// process_list_element_holder is used to make an element in ProcessList live while BACKUP is working asynchronously.
auto process_list_element = context_in_use->getProcessListElement();
/// Update context to preserve query information in processlist (settings, current_database)
process_list_element->updateContext(context_in_use);
thread_pool.scheduleOrThrowOnError(
[this,
@ -853,6 +855,8 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
/// process_list_element_holder is used to make an element in ProcessList live while RESTORE is working asynchronously.
auto process_list_element = context_in_use->getProcessListElement();
/// Update context to preserve query information in processlist (settings, current_database)
process_list_element->updateContext(context_in_use);
thread_pool.scheduleOrThrowOnError(
[this,

View File

@ -46,8 +46,8 @@ namespace
if (zookeeper_path_ast && (zookeeper_path_ast->value.getType() == Field::Types::String) &&
replica_name_ast && (replica_name_ast->value.getType() == Field::Types::String))
{
String & zookeeper_path_arg = zookeeper_path_ast->value.get<String>();
String & replica_name_arg = replica_name_ast->value.get<String>();
String & zookeeper_path_arg = zookeeper_path_ast->value.safeGet<String>();
String & replica_name_arg = replica_name_ast->value.safeGet<String>();
if (create.uuid != UUIDHelpers::Nil)
{
String table_uuid_str = toString(create.uuid);

View File

@ -31,7 +31,7 @@ namespace
{
if (field.getType() == Field::Types::String)
{
const String & str = field.get<const String &>();
const String & str = field.safeGet<const String &>();
if (str == "1" || boost::iequals(str, "true") || boost::iequals(str, "create"))
{
value = RestoreTableCreationMode::kCreate;
@ -54,7 +54,7 @@ namespace
if (field.getType() == Field::Types::UInt64)
{
UInt64 number = field.get<UInt64>();
UInt64 number = field.safeGet<UInt64>();
if (number == 1)
{
value = RestoreTableCreationMode::kCreate;
@ -95,7 +95,7 @@ namespace
{
if (field.getType() == Field::Types::String)
{
const String & str = field.get<const String &>();
const String & str = field.safeGet<const String &>();
if (str == "1" || boost::iequals(str, "true") || boost::iequals(str, "create"))
{
value = RestoreAccessCreationMode::kCreate;
@ -118,7 +118,7 @@ namespace
if (field.getType() == Field::Types::UInt64)
{
UInt64 number = field.get<UInt64>();
UInt64 number = field.safeGet<UInt64>();
if (number == 1)
{
value = RestoreAccessCreationMode::kCreate;

View File

@ -19,7 +19,7 @@ SettingFieldOptionalString::SettingFieldOptionalString(const Field & field)
if (field.getType() == Field::Types::String)
{
value = field.get<const String &>();
value = field.safeGet<const String &>();
return;
}

View File

@ -22,7 +22,7 @@ namespace ErrorCodes
if (field.getType() == Field::Types::String)
{
const String & str = field.get<const String &>();
const String & str = field.safeGet<const String &>();
if (str.empty())
{
value = std::nullopt;

View File

@ -353,8 +353,8 @@ target_link_libraries(clickhouse_common_io
Poco::Foundation
)
if (TARGET ch_contrib::fiu)
target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::fiu)
if (TARGET ch_contrib::libfiu)
target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::libfiu)
endif()
if (TARGET ch_contrib::cpuid)
@ -556,14 +556,13 @@ target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::lz4)
if (TARGET ch_contrib::qpl)
dbms_target_link_libraries(PUBLIC ch_contrib::qpl)
target_link_libraries (clickhouse_compression PUBLIC ch_contrib::qpl)
target_link_libraries (clickhouse_compression PUBLIC ch_contrib::accel-config)
endif ()
if (TARGET ch_contrib::accel-config)
dbms_target_link_libraries(PUBLIC ch_contrib::accel-config)
endif ()
if (TARGET ch_contrib::qatzstd_plugin)
if (TARGET ch_contrib::accel-config AND TARGET ch_contrib::qatzstd_plugin)
dbms_target_link_libraries(PUBLIC ch_contrib::qatzstd_plugin)
dbms_target_link_libraries(PUBLIC ch_contrib::accel-config)
target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::qatzstd_plugin)
endif ()

View File

@ -214,7 +214,7 @@ void Suggest::fillWordsFromBlock(const Block & block)
Words new_words;
new_words.reserve(rows);
for (size_t i = 0; i < rows; ++i)
new_words.emplace_back(column[i].get<String>());
new_words.emplace_back(column[i].safeGet<String>());
addWords(std::move(new_words));
}

View File

@ -457,9 +457,9 @@ MutableColumnPtr ColumnAggregateFunction::cloneEmpty() const
Field ColumnAggregateFunction::operator[](size_t n) const
{
Field field = AggregateFunctionStateData();
field.get<AggregateFunctionStateData &>().name = type_string;
field.safeGet<AggregateFunctionStateData &>().name = type_string;
{
WriteBufferFromString buffer(field.get<AggregateFunctionStateData &>().data);
WriteBufferFromString buffer(field.safeGet<AggregateFunctionStateData &>().data);
func->serialize(data[n], buffer, version);
}
return field;
@ -467,12 +467,7 @@ Field ColumnAggregateFunction::operator[](size_t n) const
void ColumnAggregateFunction::get(size_t n, Field & res) const
{
res = AggregateFunctionStateData();
res.get<AggregateFunctionStateData &>().name = type_string;
{
WriteBufferFromString buffer(res.get<AggregateFunctionStateData &>().data);
func->serialize(data[n], buffer, version);
}
res = operator[](n);
}
StringRef ColumnAggregateFunction::getDataAt(size_t n) const
@ -552,7 +547,7 @@ void ColumnAggregateFunction::insert(const Field & x)
"Inserting field of type {} into ColumnAggregateFunction. Expected {}",
x.getTypeName(), Field::Types::AggregateFunctionState);
const auto & field_name = x.get<const AggregateFunctionStateData &>().name;
const auto & field_name = x.safeGet<const AggregateFunctionStateData &>().name;
if (type_string != field_name)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot insert filed with type {} into column with type {}",
field_name, type_string);
@ -560,7 +555,7 @@ void ColumnAggregateFunction::insert(const Field & x)
ensureOwnership();
Arena & arena = createOrGetArena();
pushBackAndCreateState(data, arena, func.get());
ReadBufferFromString read_buffer(x.get<const AggregateFunctionStateData &>().data);
ReadBufferFromString read_buffer(x.safeGet<const AggregateFunctionStateData &>().data);
func->deserialize(data.back(), read_buffer, version, &arena);
}
@ -569,14 +564,14 @@ bool ColumnAggregateFunction::tryInsert(const DB::Field & x)
if (x.getType() != Field::Types::AggregateFunctionState)
return false;
const auto & field_name = x.get<const AggregateFunctionStateData &>().name;
const auto & field_name = x.safeGet<const AggregateFunctionStateData &>().name;
if (type_string != field_name)
return false;
ensureOwnership();
Arena & arena = createOrGetArena();
pushBackAndCreateState(data, arena, func.get());
ReadBufferFromString read_buffer(x.get<const AggregateFunctionStateData &>().data);
ReadBufferFromString read_buffer(x.safeGet<const AggregateFunctionStateData &>().data);
func->deserialize(data.back(), read_buffer, version, &arena);
return true;
}

View File

@ -141,7 +141,7 @@ void ColumnArray::get(size_t n, Field & res) const
size, max_array_size_as_field);
res = Array();
Array & res_arr = res.get<Array &>();
Array & res_arr = res.safeGet<Array &>();
res_arr.reserve(size);
for (size_t i = 0; i < size; ++i)
@ -309,7 +309,7 @@ void ColumnArray::updateHashFast(SipHash & hash) const
void ColumnArray::insert(const Field & x)
{
const Array & array = x.get<const Array &>();
const Array & array = x.safeGet<const Array &>();
size_t size = array.size();
for (size_t i = 0; i < size; ++i)
getData().insert(array[i]);
@ -321,7 +321,7 @@ bool ColumnArray::tryInsert(const Field & x)
if (x.getType() != Field::Types::Which::Array)
return false;
const Array & array = x.get<const Array &>();
const Array & array = x.safeGet<const Array &>();
size_t size = array.size();
for (size_t i = 0; i < size; ++i)
{
@ -452,6 +452,22 @@ void ColumnArray::reserve(size_t n)
getData().reserve(n); /// The average size of arrays is not taken into account here. Or it is considered to be no more than 1.
}
void ColumnArray::prepareForSquashing(const Columns & source_columns)
{
size_t new_size = size();
Columns source_data_columns;
source_data_columns.reserve(source_columns.size());
for (const auto & source_column : source_columns)
{
const auto & source_array_column = assert_cast<const ColumnArray &>(*source_column);
new_size += source_array_column.size();
source_data_columns.push_back(source_array_column.getDataPtr());
}
getOffsets().reserve_exact(new_size);
data->prepareForSquashing(source_data_columns);
}
void ColumnArray::shrinkToFit()
{
getOffsets().shrink_to_fit();

View File

@ -118,6 +118,7 @@ public:
void updatePermutationWithCollation(const Collator & collator, PermutationSortDirection direction, PermutationSortStability stability,
size_t limit, int nan_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override;
void reserve(size_t n) override;
void prepareForSquashing(const Columns & source_columns) override;
void shrinkToFit() override;
void ensureOwnership() override;
size_t byteSize() const override;

View File

@ -74,7 +74,7 @@ public:
void insertData(const char * src, size_t /*length*/) override;
void insertDefault() override { data.push_back(T()); }
void insertManyDefaults(size_t length) override { data.resize_fill(data.size() + length); }
void insert(const Field & x) override { data.push_back(x.get<T>()); }
void insert(const Field & x) override { data.push_back(x.safeGet<T>()); }
bool tryInsert(const Field & x) override;
#if !defined(DEBUG_OR_SANITIZER_BUILD)
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;

View File

@ -643,6 +643,116 @@ ColumnPtr ColumnDynamic::compress() const
});
}
void ColumnDynamic::prepareForSquashing(const Columns & source_columns)
{
if (source_columns.empty())
return;
/// Internal variants of source dynamic columns may differ.
/// We want to preallocate memory for all variants we will have after squashing.
/// It may happen that the total number of variants in source columns will
/// exceed the limit, in this case we will choose the most frequent variants.
/// First, preallocate memory for variant discriminators and offsets.
size_t new_size = size();
for (const auto & source_column : source_columns)
new_size += source_column->size();
auto & variant_col = getVariantColumn();
variant_col.getLocalDiscriminators().reserve_exact(new_size);
variant_col.getOffsets().reserve_exact(new_size);
/// Second, collect all variants and their total sizes.
std::unordered_map<String, size_t> total_variant_sizes;
DataTypes all_variants;
auto add_variants = [&](const ColumnDynamic & source_dynamic)
{
const auto & source_variant_column = source_dynamic.getVariantColumn();
const auto & source_variant_info = source_dynamic.getVariantInfo();
const auto & source_variants = assert_cast<const DataTypeVariant &>(*source_variant_info.variant_type).getVariants();
for (size_t i = 0; i != source_variants.size(); ++i)
{
const auto & variant_name = source_variant_info.variant_names[i];
auto it = total_variant_sizes.find(variant_name);
/// Add this variant to the list of all variants if we didn't see it yet.
if (it == total_variant_sizes.end())
{
all_variants.push_back(source_variants[i]);
it = total_variant_sizes.emplace(variant_name, 0).first;
}
it->second += source_variant_column.getVariantByGlobalDiscriminator(i).size();
}
};
for (const auto & source_column : source_columns)
add_variants(assert_cast<const ColumnDynamic &>(*source_column));
/// Add variants from this dynamic column.
add_variants(*this);
DataTypePtr result_variant_type;
/// Check if the number of all variants exceeds the limit.
if (all_variants.size() > max_dynamic_types || (all_variants.size() == max_dynamic_types && !total_variant_sizes.contains("String")))
{
/// We want to keep the most frequent variants in the resulting dynamic column.
DataTypes result_variants;
result_variants.reserve(max_dynamic_types);
/// Add variants from current variant column as we will not rewrite it.
for (const auto & variant : assert_cast<const DataTypeVariant &>(*variant_info.variant_type).getVariants())
result_variants.push_back(variant);
/// Add String variant in advance (if we didn't add it yet) as we must have it across variants when we reach the limit.
if (!variant_info.variant_name_to_discriminator.contains("String"))
result_variants.push_back(std::make_shared<DataTypeString>());
/// Create list of remaining variants with their sizes and sort it.
std::vector<std::pair<size_t, DataTypePtr>> variants_with_sizes;
variants_with_sizes.reserve(all_variants.size() - variant_info.variant_names.size());
for (const auto & variant : all_variants)
{
/// Add variant to the list only of we didn't add it yet.
auto variant_name = variant->getName();
if (variant_name != "String" && !variant_info.variant_name_to_discriminator.contains(variant_name))
variants_with_sizes.emplace_back(total_variant_sizes[variant->getName()], variant);
}
std::sort(variants_with_sizes.begin(), variants_with_sizes.end(), std::greater());
/// Add the most frequent variants until we reach max_dynamic_types.
size_t num_new_variants = max_dynamic_types - result_variants.size();
for (size_t i = 0; i != num_new_variants; ++i)
result_variants.push_back(variants_with_sizes[i].second);
result_variant_type = std::make_shared<DataTypeVariant>(result_variants);
}
else
{
result_variant_type = std::make_shared<DataTypeVariant>(all_variants);
}
if (!result_variant_type->equals(*variant_info.variant_type))
updateVariantInfoAndExpandVariantColumn(result_variant_type);
/// Now current dynamic column has all resulting variants and we can call
/// prepareForSquashing on them to preallocate the memory.
for (size_t i = 0; i != variant_info.variant_names.size(); ++i)
{
Columns source_variant_columns;
source_variant_columns.reserve(source_columns.size());
for (const auto & source_column : source_columns)
{
const auto & source_dynamic_column = assert_cast<const ColumnDynamic &>(*source_column);
const auto & source_variant_info = source_dynamic_column.getVariantInfo();
/// Try to find this variant in the current source column.
auto it = source_variant_info.variant_name_to_discriminator.find(variant_info.variant_names[i]);
if (it != source_variant_info.variant_name_to_discriminator.end())
source_variant_columns.push_back(source_dynamic_column.getVariantColumn().getVariantPtrByGlobalDiscriminator(it->second));
}
variant_col.getVariantByGlobalDiscriminator(i).prepareForSquashing(source_variant_columns);
}
}
void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source_columns)
{
if (!empty())

View File

@ -254,6 +254,8 @@ public:
variant_column->reserve(n);
}
void prepareForSquashing(const Columns & source_columns) override;
void ensureOwnership() override
{
variant_column->ensureOwnership();

View File

@ -59,7 +59,7 @@ bool ColumnFixedString::isDefaultAt(size_t index) const
void ColumnFixedString::insert(const Field & x)
{
const String & s = x.get<const String &>();
const String & s = x.safeGet<const String &>();
insertData(s.data(), s.size());
}
@ -67,7 +67,7 @@ bool ColumnFixedString::tryInsert(const Field & x)
{
if (x.getType() != Field::Types::Which::String)
return false;
const String & s = x.get<const String &>();
const String & s = x.safeGet<const String &>();
if (s.size() > n)
return false;
insertData(s.data(), s.size());

View File

@ -72,7 +72,7 @@ void ColumnMap::get(size_t n, Field & res) const
size_t size = offsets[n] - offsets[n - 1];
res = Map();
auto & map = res.get<Map &>();
auto & map = res.safeGet<Map &>();
map.reserve(size);
for (size_t i = 0; i < size; ++i)
@ -96,7 +96,7 @@ void ColumnMap::insertData(const char *, size_t)
void ColumnMap::insert(const Field & x)
{
const auto & map = x.get<const Map &>();
const auto & map = x.safeGet<const Map &>();
nested->insert(Array(map.begin(), map.end()));
}
@ -105,7 +105,7 @@ bool ColumnMap::tryInsert(const Field & x)
if (x.getType() != Field::Types::Which::Map)
return false;
const auto & map = x.get<const Map &>();
const auto & map = x.safeGet<const Map &>();
return nested->tryInsert(Array(map.begin(), map.end()));
}
@ -249,6 +249,15 @@ void ColumnMap::reserve(size_t n)
nested->reserve(n);
}
void ColumnMap::prepareForSquashing(const Columns & source_columns)
{
Columns nested_source_columns;
nested_source_columns.reserve(source_columns.size());
for (const auto & source_column : source_columns)
nested_source_columns.push_back(assert_cast<const ColumnMap &>(*source_column).getNestedColumnPtr());
nested->prepareForSquashing(nested_source_columns);
}
void ColumnMap::shrinkToFit()
{
nested->shrinkToFit();
@ -288,8 +297,8 @@ void ColumnMap::getExtremes(Field & min, Field & max) const
/// Convert result Array fields to Map fields because client expect min and max field to have type Map
Array nested_min_value = nested_min.get<Array>();
Array nested_max_value = nested_max.get<Array>();
Array nested_min_value = nested_min.safeGet<Array>();
Array nested_max_value = nested_max.safeGet<Array>();
Map map_min_value(nested_min_value.begin(), nested_min_value.end());
Map map_max_value(nested_max_value.begin(), nested_max_value.end());

View File

@ -94,6 +94,7 @@ public:
void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override;
void reserve(size_t n) override;
void prepareForSquashing(const Columns & source_columns) override;
void shrinkToFit() override;
void ensureOwnership() override;
size_t byteSize() const override;

View File

@ -706,6 +706,22 @@ void ColumnNullable::reserve(size_t n)
getNullMapData().reserve(n);
}
void ColumnNullable::prepareForSquashing(const Columns & source_columns)
{
size_t new_size = size();
Columns nested_source_columns;
nested_source_columns.reserve(source_columns.size());
for (const auto & source_column : source_columns)
{
const auto & source_nullable_column = assert_cast<const ColumnNullable &>(*source_column);
new_size += source_nullable_column.size();
nested_source_columns.push_back(source_nullable_column.getNestedColumnPtr());
}
nested_column->prepareForSquashing(nested_source_columns);
getNullMapData().reserve(new_size);
}
void ColumnNullable::shrinkToFit()
{
getNestedColumn().shrinkToFit();

View File

@ -125,6 +125,7 @@ public:
size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override;
size_t estimateCardinalityInPermutedRange(const Permutation & permutation, const EqualRange & equal_range) const override;
void reserve(size_t n) override;
void prepareForSquashing(const Columns & source_columns) override;
void shrinkToFit() override;
void ensureOwnership() override;
size_t byteSize() const override;

View File

@ -698,7 +698,7 @@ void ColumnObject::forEachSubcolumnRecursively(RecursiveMutableColumnCallback ca
void ColumnObject::insert(const Field & field)
{
const auto & object = field.get<const Object &>();
const auto & object = field.safeGet<const Object &>();
HashSet<StringRef, StringRefHash> inserted_paths;
size_t old_size = size();
@ -754,7 +754,7 @@ void ColumnObject::get(size_t n, Field & res) const
{
assert(n < size());
res = Object();
auto & object = res.get<Object &>();
auto & object = res.safeGet<Object &>();
for (const auto & entry : subcolumns)
{

View File

@ -557,6 +557,21 @@ void ColumnString::reserve(size_t n)
offsets.reserve_exact(n);
}
void ColumnString::prepareForSquashing(const Columns & source_columns)
{
size_t new_size = size();
size_t new_chars_size = chars.size();
for (const auto & source_column : source_columns)
{
const auto & source_string_column = assert_cast<const ColumnString &>(*source_column);
new_size += source_string_column.size();
new_chars_size += source_string_column.chars.size();
}
offsets.reserve_exact(new_size);
chars.reserve_exact(new_chars_size);
}
void ColumnString::shrinkToFit()
{
chars.shrink_to_fit();

View File

@ -123,7 +123,7 @@ public:
void insert(const Field & x) override
{
const String & s = x.get<const String &>();
const String & s = x.safeGet<const String &>();
const size_t old_size = chars.size();
const size_t size_to_append = s.size() + 1;
const size_t new_size = old_size + size_to_append;
@ -283,6 +283,7 @@ public:
ColumnPtr compress() const override;
void reserve(size_t n) override;
void prepareForSquashing(const Columns & source_columns) override;
void shrinkToFit() override;
void getExtremes(Field & min, Field & max) const override;

View File

@ -141,7 +141,7 @@ void ColumnTuple::get(size_t n, Field & res) const
const size_t tuple_size = columns.size();
res = Tuple();
Tuple & res_tuple = res.get<Tuple &>();
Tuple & res_tuple = res.safeGet<Tuple &>();
res_tuple.reserve(tuple_size);
for (size_t i = 0; i < tuple_size; ++i)
@ -169,7 +169,7 @@ void ColumnTuple::insertData(const char *, size_t)
void ColumnTuple::insert(const Field & x)
{
const auto & tuple = x.get<const Tuple &>();
const auto & tuple = x.safeGet<const Tuple &>();
const size_t tuple_size = columns.size();
if (tuple.size() != tuple_size)
@ -185,7 +185,7 @@ bool ColumnTuple::tryInsert(const Field & x)
if (x.getType() != Field::Types::Which::Tuple)
return false;
const auto & tuple = x.get<const Tuple &>();
const auto & tuple = x.safeGet<const Tuple &>();
const size_t tuple_size = columns.size();
if (tuple.size() != tuple_size)
@ -595,6 +595,19 @@ void ColumnTuple::reserve(size_t n)
getColumn(i).reserve(n);
}
void ColumnTuple::prepareForSquashing(const Columns & source_columns)
{
const size_t tuple_size = columns.size();
for (size_t i = 0; i < tuple_size; ++i)
{
Columns nested_columns;
nested_columns.reserve(source_columns.size());
for (const auto & source_column : source_columns)
nested_columns.push_back(assert_cast<const ColumnTuple &>(*source_column).getColumnPtr(i));
getColumn(i).prepareForSquashing(nested_columns);
}
}
void ColumnTuple::shrinkToFit()
{
const size_t tuple_size = columns.size();

View File

@ -110,6 +110,7 @@ public:
void updatePermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges& equal_ranges) const override;
void reserve(size_t n) override;
void prepareForSquashing(const Columns & source_columns) override;
void shrinkToFit() override;
void ensureOwnership() override;
size_t byteSize() const override;

View File

@ -1247,8 +1247,25 @@ void ColumnVariant::updatePermutation(IColumn::PermutationSortDirection directio
void ColumnVariant::reserve(size_t n)
{
local_discriminators->reserve(n);
offsets->reserve(n);
getLocalDiscriminators().reserve_exact(n);
getOffsets().reserve_exact(n);
}
void ColumnVariant::prepareForSquashing(const Columns & source_columns)
{
size_t new_size = size();
for (const auto & source_column : source_columns)
new_size += source_column->size();
reserve(new_size);
for (size_t i = 0; i != variants.size(); ++i)
{
Columns source_variant_columns;
source_variant_columns.reserve(source_columns.size());
for (const auto & source_column : source_columns)
source_variant_columns.push_back(assert_cast<const ColumnVariant &>(*source_column).getVariantPtrByGlobalDiscriminator(i));
getVariantByGlobalDiscriminator(i).prepareForSquashing(source_variant_columns);
}
}
void ColumnVariant::ensureOwnership()

View File

@ -237,6 +237,7 @@ public:
size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override;
void reserve(size_t n) override;
void prepareForSquashing(const Columns & source_columns) override;
void ensureOwnership() override;
size_t byteSize() const override;
size_t byteSizeAt(size_t n) const override;

View File

@ -85,7 +85,7 @@ public:
void insertMany(const Field & field, size_t length) override
{
data.resize_fill(data.size() + length, static_cast<T>(field.get<T>()));
data.resize_fill(data.size() + length, static_cast<T>(field.safeGet<T>()));
}
void insertData(const char * pos, size_t) override
@ -235,7 +235,7 @@ public:
void insert(const Field & x) override
{
data.push_back(static_cast<T>(x.get<T>()));
data.push_back(static_cast<T>(x.safeGet<T>()));
}
bool tryInsert(const DB::Field & x) override;

View File

@ -475,6 +475,15 @@ public:
/// It affects performance only (not correctness).
virtual void reserve(size_t /*n*/) {}
/// Reserve memory before squashing all specified source columns into this column.
virtual void prepareForSquashing(const std::vector<Ptr> & source_columns)
{
size_t new_size = size();
for (const auto & source_column : source_columns)
new_size += source_column->size();
reserve(new_size);
}
/// Requests the removal of unused capacity.
/// It is a non-binding request to reduce the capacity of the underlying container to its size.
virtual void shrinkToFit() {}

View File

@ -108,10 +108,10 @@ void checkColumnVariant1(ColumnVariant * column)
ASSERT_EQ(offsets[1], 0);
ASSERT_EQ(offsets[3], 1);
ASSERT_TRUE(column->isDefaultAt(2) && column->isDefaultAt(4));
ASSERT_EQ((*column)[0].get<UInt32>(), 42);
ASSERT_EQ((*column)[1].get<String>(), "Hello");
ASSERT_EQ((*column)[0].safeGet<UInt32>(), 42);
ASSERT_EQ((*column)[1].safeGet<String>(), "Hello");
ASSERT_TRUE((*column)[2].isNull());
ASSERT_EQ((*column)[3].get<String>(), "World");
ASSERT_EQ((*column)[3].safeGet<String>(), "World");
ASSERT_TRUE((*column)[4].isNull());
}
@ -209,9 +209,9 @@ TEST(ColumnVariant, CreateFromDiscriminatorsAndOneFullColumnNoNulls)
ASSERT_EQ(offsets[0], 0);
ASSERT_EQ(offsets[1], 1);
ASSERT_EQ(offsets[2], 2);
ASSERT_EQ((*column)[0].get<UInt64>(), 0);
ASSERT_EQ((*column)[1].get<UInt64>(), 1);
ASSERT_EQ((*column)[2].get<UInt64>(), 2);
ASSERT_EQ((*column)[0].safeGet<UInt64>(), 0);
ASSERT_EQ((*column)[1].safeGet<UInt64>(), 1);
ASSERT_EQ((*column)[2].safeGet<UInt64>(), 2);
}
TEST(ColumnVariant, CreateFromDiscriminatorsAndOneFullColumnNoNullsWithLocalOrder)
@ -222,9 +222,9 @@ TEST(ColumnVariant, CreateFromDiscriminatorsAndOneFullColumnNoNullsWithLocalOrde
ASSERT_EQ(offsets[0], 0);
ASSERT_EQ(offsets[1], 1);
ASSERT_EQ(offsets[2], 2);
ASSERT_EQ((*column)[0].get<UInt64>(), 0);
ASSERT_EQ((*column)[1].get<UInt64>(), 1);
ASSERT_EQ((*column)[2].get<UInt64>(), 2);
ASSERT_EQ((*column)[0].safeGet<UInt64>(), 0);
ASSERT_EQ((*column)[1].safeGet<UInt64>(), 1);
ASSERT_EQ((*column)[2].safeGet<UInt64>(), 2);
ASSERT_EQ(column->localDiscriminatorAt(0), 2);
ASSERT_EQ(column->localDiscriminatorAt(1), 2);
ASSERT_EQ(column->localDiscriminatorAt(2), 2);
@ -331,9 +331,9 @@ TEST(ColumnVariant, CloneResizedGeneral1)
ASSERT_EQ(offsets[0], 0);
ASSERT_EQ(offsets[1], 0);
ASSERT_EQ(offsets[3], 1);
ASSERT_EQ((*resized_column_variant)[0].get<UInt64>(), 42);
ASSERT_EQ((*resized_column_variant)[1].get<String>(), "Hello");
ASSERT_EQ((*resized_column_variant)[3].get<UInt64>(), 43);
ASSERT_EQ((*resized_column_variant)[0].safeGet<UInt64>(), 42);
ASSERT_EQ((*resized_column_variant)[1].safeGet<String>(), "Hello");
ASSERT_EQ((*resized_column_variant)[3].safeGet<UInt64>(), 43);
}
TEST(ColumnVariant, CloneResizedGeneral2)
@ -367,7 +367,7 @@ TEST(ColumnVariant, CloneResizedGeneral2)
ASSERT_EQ(discriminators[2], ColumnVariant::NULL_DISCRIMINATOR);
const auto & offsets = resized_column_variant->getOffsets();
ASSERT_EQ(offsets[0], 0);
ASSERT_EQ((*resized_column_variant)[0].get<UInt64>(), 42);
ASSERT_EQ((*resized_column_variant)[0].safeGet<UInt64>(), 42);
}
TEST(ColumnVariant, CloneResizedGeneral3)
@ -405,10 +405,10 @@ TEST(ColumnVariant, CloneResizedGeneral3)
ASSERT_EQ(offsets[1], 0);
ASSERT_EQ(offsets[2], 1);
ASSERT_EQ(offsets[3], 1);
ASSERT_EQ((*resized_column_variant)[0].get<UInt64>(), 42);
ASSERT_EQ((*resized_column_variant)[1].get<String>(), "Hello");
ASSERT_EQ((*resized_column_variant)[2].get<String>(), "World");
ASSERT_EQ((*resized_column_variant)[3].get<UInt64>(), 43);
ASSERT_EQ((*resized_column_variant)[0].safeGet<UInt64>(), 42);
ASSERT_EQ((*resized_column_variant)[1].safeGet<String>(), "Hello");
ASSERT_EQ((*resized_column_variant)[2].safeGet<String>(), "World");
ASSERT_EQ((*resized_column_variant)[3].safeGet<UInt64>(), 43);
}
MutableColumnPtr createDiscriminators2()
@ -465,7 +465,7 @@ TEST(ColumnVariant, InsertFrom)
auto column_from = createVariantColumn2(change_order);
column_to->insertFrom(*column_from, 3);
ASSERT_EQ(column_to->globalDiscriminatorAt(5), 0);
ASSERT_EQ((*column_to)[5].get<UInt64>(), 43);
ASSERT_EQ((*column_to)[5].safeGet<UInt64>(), 43);
}
}
@ -478,8 +478,8 @@ TEST(ColumnVariant, InsertRangeFromOneColumnNoNulls)
column_to->insertRangeFrom(*column_from, 2, 2);
ASSERT_EQ(column_to->globalDiscriminatorAt(7), 0);
ASSERT_EQ(column_to->globalDiscriminatorAt(8), 0);
ASSERT_EQ((*column_to)[7].get<UInt64>(), 2);
ASSERT_EQ((*column_to)[8].get<UInt64>(), 3);
ASSERT_EQ((*column_to)[7].safeGet<UInt64>(), 2);
ASSERT_EQ((*column_to)[8].safeGet<UInt64>(), 3);
}
}
@ -494,9 +494,9 @@ TEST(ColumnVariant, InsertRangeFromGeneral)
ASSERT_EQ(column_to->globalDiscriminatorAt(6), ColumnVariant::NULL_DISCRIMINATOR);
ASSERT_EQ(column_to->globalDiscriminatorAt(7), 0);
ASSERT_EQ(column_to->globalDiscriminatorAt(8), 1);
ASSERT_EQ((*column_to)[5].get<String>(), "Hello");
ASSERT_EQ((*column_to)[7].get<UInt64>(), 43);
ASSERT_EQ((*column_to)[8].get<String>(), "World");
ASSERT_EQ((*column_to)[5].safeGet<String>(), "Hello");
ASSERT_EQ((*column_to)[7].safeGet<UInt64>(), 43);
ASSERT_EQ((*column_to)[8].safeGet<String>(), "World");
}
}
@ -509,8 +509,8 @@ TEST(ColumnVariant, InsertManyFrom)
column_to->insertManyFrom(*column_from, 3, 2);
ASSERT_EQ(column_to->globalDiscriminatorAt(5), 0);
ASSERT_EQ(column_to->globalDiscriminatorAt(6), 0);
ASSERT_EQ((*column_to)[5].get<UInt64>(), 43);
ASSERT_EQ((*column_to)[6].get<UInt64>(), 43);
ASSERT_EQ((*column_to)[5].safeGet<UInt64>(), 43);
ASSERT_EQ((*column_to)[6].safeGet<UInt64>(), 43);
}
}
@ -520,8 +520,8 @@ TEST(ColumnVariant, PopBackOneColumnNoNulls)
column->popBack(3);
ASSERT_EQ(column->size(), 2);
ASSERT_EQ(column->getVariantByLocalDiscriminator(0).size(), 2);
ASSERT_EQ((*column)[0].get<UInt64>(), 0);
ASSERT_EQ((*column)[1].get<UInt64>(), 1);
ASSERT_EQ((*column)[0].safeGet<UInt64>(), 0);
ASSERT_EQ((*column)[1].safeGet<UInt64>(), 1);
}
TEST(ColumnVariant, PopBackGeneral)
@ -531,8 +531,8 @@ TEST(ColumnVariant, PopBackGeneral)
ASSERT_EQ(column->size(), 3);
ASSERT_EQ(column->getVariantByLocalDiscriminator(0).size(), 1);
ASSERT_EQ(column->getVariantByLocalDiscriminator(1).size(), 1);
ASSERT_EQ((*column)[0].get<UInt64>(), 42);
ASSERT_EQ((*column)[1].get<String>(), "Hello");
ASSERT_EQ((*column)[0].safeGet<UInt64>(), 42);
ASSERT_EQ((*column)[1].safeGet<String>(), "Hello");
ASSERT_TRUE((*column)[2].isNull());
}
@ -545,8 +545,8 @@ TEST(ColumnVariant, FilterOneColumnNoNulls)
filter.push_back(1);
auto filtered_column = column->filter(filter, -1);
ASSERT_EQ(filtered_column->size(), 2);
ASSERT_EQ((*filtered_column)[0].get<UInt64>(), 0);
ASSERT_EQ((*filtered_column)[1].get<UInt64>(), 2);
ASSERT_EQ((*filtered_column)[0].safeGet<UInt64>(), 0);
ASSERT_EQ((*filtered_column)[1].safeGet<UInt64>(), 2);
}
TEST(ColumnVariant, FilterGeneral)
@ -562,7 +562,7 @@ TEST(ColumnVariant, FilterGeneral)
filter.push_back(0);
auto filtered_column = column->filter(filter, -1);
ASSERT_EQ(filtered_column->size(), 3);
ASSERT_EQ((*filtered_column)[0].get<String>(), "Hello");
ASSERT_EQ((*filtered_column)[0].safeGet<String>(), "Hello");
ASSERT_TRUE((*filtered_column)[1].isNull());
ASSERT_TRUE((*filtered_column)[2].isNull());
}
@ -577,9 +577,9 @@ TEST(ColumnVariant, PermuteAndIndexOneColumnNoNulls)
permutation.push_back(0);
auto permuted_column = column->permute(permutation, 3);
ASSERT_EQ(permuted_column->size(), 3);
ASSERT_EQ((*permuted_column)[0].get<UInt64>(), 1);
ASSERT_EQ((*permuted_column)[1].get<UInt64>(), 3);
ASSERT_EQ((*permuted_column)[2].get<UInt64>(), 2);
ASSERT_EQ((*permuted_column)[0].safeGet<UInt64>(), 1);
ASSERT_EQ((*permuted_column)[1].safeGet<UInt64>(), 3);
ASSERT_EQ((*permuted_column)[2].safeGet<UInt64>(), 2);
auto index = ColumnUInt64::create();
index->getData().push_back(1);
@ -588,9 +588,9 @@ TEST(ColumnVariant, PermuteAndIndexOneColumnNoNulls)
index->getData().push_back(0);
auto indexed_column = column->index(*index, 3);
ASSERT_EQ(indexed_column->size(), 3);
ASSERT_EQ((*indexed_column)[0].get<UInt64>(), 1);
ASSERT_EQ((*indexed_column)[1].get<UInt64>(), 3);
ASSERT_EQ((*indexed_column)[2].get<UInt64>(), 2);
ASSERT_EQ((*indexed_column)[0].safeGet<UInt64>(), 1);
ASSERT_EQ((*indexed_column)[1].safeGet<UInt64>(), 3);
ASSERT_EQ((*indexed_column)[2].safeGet<UInt64>(), 2);
}
TEST(ColumnVariant, PermuteGeneral)
@ -603,9 +603,9 @@ TEST(ColumnVariant, PermuteGeneral)
permutation.push_back(5);
auto permuted_column = column->permute(permutation, 4);
ASSERT_EQ(permuted_column->size(), 4);
ASSERT_EQ((*permuted_column)[0].get<UInt64>(), 43);
ASSERT_EQ((*permuted_column)[1].get<String>(), "World");
ASSERT_EQ((*permuted_column)[2].get<String>(), "Hello");
ASSERT_EQ((*permuted_column)[0].safeGet<UInt64>(), 43);
ASSERT_EQ((*permuted_column)[1].safeGet<String>(), "World");
ASSERT_EQ((*permuted_column)[2].safeGet<String>(), "Hello");
ASSERT_TRUE((*permuted_column)[3].isNull());
}
@ -618,12 +618,12 @@ TEST(ColumnVariant, ReplicateOneColumnNoNull)
offsets.push_back(6);
auto replicated_column = column->replicate(offsets);
ASSERT_EQ(replicated_column->size(), 6);
ASSERT_EQ((*replicated_column)[0].get<UInt64>(), 1);
ASSERT_EQ((*replicated_column)[1].get<UInt64>(), 1);
ASSERT_EQ((*replicated_column)[2].get<UInt64>(), 1);
ASSERT_EQ((*replicated_column)[3].get<UInt64>(), 2);
ASSERT_EQ((*replicated_column)[4].get<UInt64>(), 2);
ASSERT_EQ((*replicated_column)[5].get<UInt64>(), 2);
ASSERT_EQ((*replicated_column)[0].safeGet<UInt64>(), 1);
ASSERT_EQ((*replicated_column)[1].safeGet<UInt64>(), 1);
ASSERT_EQ((*replicated_column)[2].safeGet<UInt64>(), 1);
ASSERT_EQ((*replicated_column)[3].safeGet<UInt64>(), 2);
ASSERT_EQ((*replicated_column)[4].safeGet<UInt64>(), 2);
ASSERT_EQ((*replicated_column)[5].safeGet<UInt64>(), 2);
}
TEST(ColumnVariant, ReplicateGeneral)
@ -637,9 +637,9 @@ TEST(ColumnVariant, ReplicateGeneral)
offsets.push_back(7);
auto replicated_column = column->replicate(offsets);
ASSERT_EQ(replicated_column->size(), 7);
ASSERT_EQ((*replicated_column)[0].get<UInt64>(), 42);
ASSERT_EQ((*replicated_column)[1].get<String>(), "Hello");
ASSERT_EQ((*replicated_column)[2].get<String>(), "Hello");
ASSERT_EQ((*replicated_column)[0].safeGet<UInt64>(), 42);
ASSERT_EQ((*replicated_column)[1].safeGet<String>(), "Hello");
ASSERT_EQ((*replicated_column)[2].safeGet<String>(), "Hello");
ASSERT_TRUE((*replicated_column)[3].isNull());
ASSERT_TRUE((*replicated_column)[4].isNull());
ASSERT_TRUE((*replicated_column)[5].isNull());
@ -657,13 +657,13 @@ TEST(ColumnVariant, ScatterOneColumnNoNulls)
selector.push_back(1);
auto columns = column->scatter(3, selector);
ASSERT_EQ(columns[0]->size(), 2);
ASSERT_EQ((*columns[0])[0].get<UInt64>(), 0);
ASSERT_EQ((*columns[0])[1].get<UInt64>(), 3);
ASSERT_EQ((*columns[0])[0].safeGet<UInt64>(), 0);
ASSERT_EQ((*columns[0])[1].safeGet<UInt64>(), 3);
ASSERT_EQ(columns[1]->size(), 2);
ASSERT_EQ((*columns[1])[0].get<UInt64>(), 1);
ASSERT_EQ((*columns[1])[1].get<UInt64>(), 4);
ASSERT_EQ((*columns[1])[0].safeGet<UInt64>(), 1);
ASSERT_EQ((*columns[1])[1].safeGet<UInt64>(), 4);
ASSERT_EQ(columns[2]->size(), 1);
ASSERT_EQ((*columns[2])[0].get<UInt64>(), 2);
ASSERT_EQ((*columns[2])[0].safeGet<UInt64>(), 2);
}
TEST(ColumnVariant, ScatterGeneral)
@ -680,12 +680,12 @@ TEST(ColumnVariant, ScatterGeneral)
auto columns = column->scatter(3, selector);
ASSERT_EQ(columns[0]->size(), 3);
ASSERT_EQ((*columns[0])[0].get<UInt64>(), 42);
ASSERT_EQ((*columns[0])[1].get<String>(), "Hello");
ASSERT_EQ((*columns[0])[2].get<UInt64>(), 43);
ASSERT_EQ((*columns[0])[0].safeGet<UInt64>(), 42);
ASSERT_EQ((*columns[0])[1].safeGet<String>(), "Hello");
ASSERT_EQ((*columns[0])[2].safeGet<UInt64>(), 43);
ASSERT_EQ(columns[1]->size(), 2);
ASSERT_EQ((*columns[1])[0].get<String>(), "World");
ASSERT_EQ((*columns[1])[1].get<UInt64>(), 44);
ASSERT_EQ((*columns[1])[0].safeGet<String>(), "World");
ASSERT_EQ((*columns[1])[1].safeGet<UInt64>(), 44);
ASSERT_EQ(columns[2]->size(), 2);
ASSERT_TRUE((*columns[2])[0].isNull());
ASSERT_TRUE((*columns[2])[1].isNull());

View File

@ -20,13 +20,13 @@ void testLowCardinalityNumberInsert(const DataTypePtr & data_type)
Field value;
column->get(0, value);
ASSERT_EQ(value.get<T>(), 15);
ASSERT_EQ(value.safeGet<T>(), 15);
column->get(1, value);
ASSERT_EQ(value.get<T>(), 20);
ASSERT_EQ(value.safeGet<T>(), 20);
column->get(2, value);
ASSERT_EQ(value.get<T>(), 25);
ASSERT_EQ(value.safeGet<T>(), 25);
}
TEST(ColumnLowCardinality, Insert)

View File

@ -168,7 +168,7 @@ private:
records.emplace(it, type_idx, item);
}
Records::const_iterator getImpl(std::type_index type_idx) const
typename Records::const_iterator getImpl(std::type_index type_idx) const
{
auto it = std::lower_bound(records.cbegin(), records.cend(), type_idx);

View File

@ -7,6 +7,8 @@
#include <condition_variable>
#include <mutex>
#include "config.h"
namespace DB
{
@ -15,7 +17,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
};
#if FIU_ENABLE
#if USE_LIBFIU
static struct InitFiu
{
InitFiu()
@ -135,7 +137,7 @@ void FailPointInjection::pauseFailPoint(const String & fail_point_name)
void FailPointInjection::enableFailPoint(const String & fail_point_name)
{
#if FIU_ENABLE
#if USE_LIBFIU
#define SUB_M(NAME, flags, pause) \
if (fail_point_name == FailPoints::NAME) \
{ \

View File

@ -1,17 +1,16 @@
#pragma once
#include "config.h"
#include <Common/Exception.h>
#include <Core/Types.h>
#include <Poco/Util/AbstractConfiguration.h>
#include "config.h"
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wdocumentation"
#pragma clang diagnostic ignored "-Wreserved-macro-identifier"
#include <fiu.h>
#include <fiu-control.h>
# include <fiu.h>
# include <fiu-control.h>
#pragma clang diagnostic pop
#include <unordered_map>

View File

@ -208,7 +208,7 @@ void FieldVisitorEncodeBinary::operator() (const Map & x, WriteBuffer & buf) con
writeVarUInt(size, buf);
for (size_t i = 0; i < size; ++i)
{
const Tuple & key_and_value = x[i].get<Tuple>();
const Tuple & key_and_value = x[i].safeGet<Tuple>();
Field::dispatch([&buf] (const auto & value) { FieldVisitorEncodeBinary()(value, buf); }, key_and_value[0]);
Field::dispatch([&buf] (const auto & value) { FieldVisitorEncodeBinary()(value, buf); }, key_and_value[1]);
}

View File

@ -19,7 +19,7 @@ bool FieldVisitorSum::operator() (UInt64 & x) const
return x != 0;
}
bool FieldVisitorSum::operator() (Float64 & x) const { x += rhs.get<Float64>(); return x != 0; }
bool FieldVisitorSum::operator() (Float64 & x) const { x += rhs.safeGet<Float64>(); return x != 0; }
bool FieldVisitorSum::operator() (Null &) const
{

View File

@ -37,7 +37,7 @@ public:
template <typename T>
bool operator() (DecimalField<T> & x) const
{
x += rhs.get<DecimalField<T>>();
x += rhs.safeGet<DecimalField<T>>();
return x.getValue() != T(0);
}

View File

@ -172,7 +172,7 @@ String FieldVisitorToString::operator() (const Object & x) const
String convertFieldToString(const Field & field)
{
if (field.getType() == Field::Types::Which::String)
return field.get<String>();
return field.safeGet<String>();
return applyVisitor(FieldVisitorToString(), field);
}

View File

@ -297,7 +297,7 @@ public:
}
/// Only inserts the value if key isn't already present
void ALWAYS_INLINE insertIfNotPresent(const Key & x, const Cell::Mapped & value)
void ALWAYS_INLINE insertIfNotPresent(const Key & x, const typename Cell::Mapped & value)
{
LookupResult it;
bool inserted;

View File

@ -132,7 +132,7 @@ Field QueryFuzzer::fuzzField(Field field)
if (type == Field::Types::String)
{
auto & str = field.get<std::string>();
auto & str = field.safeGet<std::string>();
UInt64 action = fuzz_rand() % 10;
switch (action)
{
@ -158,7 +158,7 @@ Field QueryFuzzer::fuzzField(Field field)
}
else if (type == Field::Types::Array)
{
auto & arr = field.get<Array>();
auto & arr = field.safeGet<Array>();
if (fuzz_rand() % 5 == 0 && !arr.empty())
{
@ -193,7 +193,7 @@ Field QueryFuzzer::fuzzField(Field field)
}
else if (type == Field::Types::Tuple)
{
auto & arr = field.get<Tuple>();
auto & arr = field.safeGet<Tuple>();
if (fuzz_rand() % 5 == 0 && !arr.empty())
{
@ -922,17 +922,17 @@ ASTPtr QueryFuzzer::fuzzLiteralUnderExpressionList(ASTPtr child)
auto type = l->value.getType();
if (type == Field::Types::Which::String && fuzz_rand() % 7 == 0)
{
String value = l->value.get<String>();
String value = l->value.safeGet<String>();
child = makeASTFunction(
"toFixedString", std::make_shared<ASTLiteral>(value), std::make_shared<ASTLiteral>(static_cast<UInt64>(value.size())));
}
else if (type == Field::Types::Which::UInt64 && fuzz_rand() % 7 == 0)
{
child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toUInt128" : "toUInt256", std::make_shared<ASTLiteral>(l->value.get<UInt64>()));
child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toUInt128" : "toUInt256", std::make_shared<ASTLiteral>(l->value.safeGet<UInt64>()));
}
else if (type == Field::Types::Which::Int64 && fuzz_rand() % 7 == 0)
{
child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toInt128" : "toInt256", std::make_shared<ASTLiteral>(l->value.get<Int64>()));
child = makeASTFunction(fuzz_rand() % 2 == 0 ? "toInt128" : "toInt256", std::make_shared<ASTLiteral>(l->value.safeGet<Int64>()));
}
else if (type == Field::Types::Which::Float64 && fuzz_rand() % 7 == 0)
{
@ -940,22 +940,22 @@ ASTPtr QueryFuzzer::fuzzLiteralUnderExpressionList(ASTPtr child)
if (decimal == 0)
child = makeASTFunction(
"toDecimal32",
std::make_shared<ASTLiteral>(l->value.get<Float64>()),
std::make_shared<ASTLiteral>(l->value.safeGet<Float64>()),
std::make_shared<ASTLiteral>(static_cast<UInt64>(fuzz_rand() % 9)));
else if (decimal == 1)
child = makeASTFunction(
"toDecimal64",
std::make_shared<ASTLiteral>(l->value.get<Float64>()),
std::make_shared<ASTLiteral>(l->value.safeGet<Float64>()),
std::make_shared<ASTLiteral>(static_cast<UInt64>(fuzz_rand() % 18)));
else if (decimal == 2)
child = makeASTFunction(
"toDecimal128",
std::make_shared<ASTLiteral>(l->value.get<Float64>()),
std::make_shared<ASTLiteral>(l->value.safeGet<Float64>()),
std::make_shared<ASTLiteral>(static_cast<UInt64>(fuzz_rand() % 38)));
else
child = makeASTFunction(
"toDecimal256",
std::make_shared<ASTLiteral>(l->value.get<Float64>()),
std::make_shared<ASTLiteral>(l->value.safeGet<Float64>()),
std::make_shared<ASTLiteral>(static_cast<UInt64>(fuzz_rand() % 76)));
}

View File

@ -27,12 +27,14 @@
#include <Common/logger_useful.h>
#include <base/scope_guard.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TIMEOUT_EXCEEDED;
extern const int ABORTED;
}
ISystemLog::~ISystemLog() = default;
@ -86,32 +88,18 @@ void SystemLogQueue<LogElement>::push(LogElement&& element)
// by one, under exclusive lock, so we will see each message count.
// It is enough to only wake the flushing thread once, after the message
// count increases past half available size.
const uint64_t queue_end = queue_front_index + queue.size();
requested_flush_up_to = std::max(requested_flush_up_to, queue_end);
flush_event.notify_all();
const auto last_log_index = queue_front_index + queue.size();
notifyFlushUnlocked(last_log_index, /* should_prepare_tables_anyway */ false);
}
if (queue.size() >= settings.max_size_rows)
{
chassert(queue.size() == settings.max_size_rows);
// Ignore all further entries until the queue is flushed.
// Log a message about that. Don't spam it -- this might be especially
// problematic in case of trace log. Remember what the front index of the
// queue was when we last logged the message. If it changed, it means the
// queue was flushed, and we can log again.
if (queue_front_index != logged_queue_full_at_index)
{
logged_queue_full_at_index = queue_front_index;
// TextLog sets its logger level to 0, so this log is a noop and
// there is no recursive logging.
lock.unlock();
LOG_ERROR(log, "Queue is full for system log '{}' at {}. max_size_rows {}",
demangle(typeid(*this).name()),
queue_front_index,
settings.max_size_rows);
}
// To the next batch we add a log message about how much we have lost
++ignored_logs;
return;
}
@ -127,20 +115,50 @@ template <typename LogElement>
void SystemLogQueue<LogElement>::handleCrash()
{
if (settings.notify_flush_on_crash)
notifyFlush(/* force */ true);
{
notifyFlush(getLastLogIndex(), /* should_prepare_tables_anyway */ true);
}
}
template <typename LogElement>
void SystemLogQueue<LogElement>::waitFlush(uint64_t expected_flushed_up_to)
void SystemLogQueue<LogElement>::notifyFlushUnlocked(Index expected_flushed_index, bool should_prepare_tables_anyway)
{
if (should_prepare_tables_anyway)
requested_prepare_tables = std::max(requested_prepare_tables, expected_flushed_index);
requested_flush_index = std::max(requested_flush_index, expected_flushed_index);
flush_event.notify_all();
}
template <typename LogElement>
void SystemLogQueue<LogElement>::notifyFlush(SystemLogQueue<LogElement>::Index expected_flushed_index, bool should_prepare_tables_anyway)
{
std::lock_guard lock(mutex);
notifyFlushUnlocked(expected_flushed_index, should_prepare_tables_anyway);
}
template <typename LogElement>
void SystemLogQueue<LogElement>::waitFlush(SystemLogQueue<LogElement>::Index expected_flushed_index, bool should_prepare_tables_anyway)
{
LOG_DEBUG(log, "Requested flush up to offset {}", expected_flushed_index);
// Use an arbitrary timeout to avoid endless waiting. 60s proved to be
// too fast for our parallel functional tests, probably because they
// heavily load the disk.
const int timeout_seconds = 180;
std::unique_lock lock(mutex);
bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&]
// there is no obligation to call notifyFlush before waitFlush, than we have to be sure that flush_event has been triggered before we wait the result
notifyFlushUnlocked(expected_flushed_index, should_prepare_tables_anyway);
auto result = confirm_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&]
{
return flushed_up_to >= expected_flushed_up_to && !is_force_prepare_tables;
if (should_prepare_tables_anyway)
return (flushed_index >= expected_flushed_index && prepared_tables >= requested_prepare_tables) || is_shutdown;
else
return (flushed_index >= expected_flushed_index) || is_shutdown;
});
if (!result)
@ -148,67 +166,63 @@ void SystemLogQueue<LogElement>::waitFlush(uint64_t expected_flushed_up_to)
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Timeout exceeded ({} s) while flushing system log '{}'.",
toString(timeout_seconds), demangle(typeid(*this).name()));
}
}
template <typename LogElement>
uint64_t SystemLogQueue<LogElement>::notifyFlush(bool should_prepare_tables_anyway)
{
uint64_t this_thread_requested_offset;
if (is_shutdown)
{
std::lock_guard lock(mutex);
if (is_shutdown)
return uint64_t(-1);
this_thread_requested_offset = queue_front_index + queue.size();
// Publish our flush request, taking care not to overwrite the requests
// made by other threads.
is_force_prepare_tables |= should_prepare_tables_anyway;
requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset);
flush_event.notify_all();
throw Exception(ErrorCodes::ABORTED, "Shutdown has been called while flushing system log '{}'. Aborting.",
demangle(typeid(*this).name()));
}
LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset);
return this_thread_requested_offset;
}
template <typename LogElement>
void SystemLogQueue<LogElement>::confirm(uint64_t to_flush_end)
SystemLogQueue<LogElement>::Index SystemLogQueue<LogElement>::getLastLogIndex()
{
std::lock_guard lock(mutex);
flushed_up_to = to_flush_end;
is_force_prepare_tables = false;
flush_event.notify_all();
return queue_front_index + queue.size();
}
template <typename LogElement>
typename SystemLogQueue<LogElement>::Index SystemLogQueue<LogElement>::pop(std::vector<LogElement> & output,
bool & should_prepare_tables_anyway,
bool & exit_this_thread)
void SystemLogQueue<LogElement>::confirm(SystemLogQueue<LogElement>::Index last_flashed_index)
{
/// Call dtors and deallocate strings without holding the global lock
output.resize(0);
std::lock_guard lock(mutex);
prepared_tables = std::max(prepared_tables, last_flashed_index);
flushed_index = std::max(flushed_index, last_flashed_index);
confirm_event.notify_all();
}
std::unique_lock lock(mutex);
flush_event.wait_for(lock,
std::chrono::milliseconds(settings.flush_interval_milliseconds),
[&] ()
template <typename LogElement>
typename SystemLogQueue<LogElement>::PopResult SystemLogQueue<LogElement>::pop()
{
PopResult result;
size_t prev_ignored_logs = 0;
{
std::unique_lock lock(mutex);
flush_event.wait_for(lock, std::chrono::milliseconds(settings.flush_interval_milliseconds), [&] ()
{
return requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables;
}
);
return requested_flush_index > flushed_index || requested_prepare_tables > prepared_tables || is_shutdown;
});
queue_front_index += queue.size();
// Swap with existing array from previous flush, to save memory
// allocations.
queue.swap(output);
if (is_shutdown)
return PopResult{.is_shutdown = true};
should_prepare_tables_anyway = is_force_prepare_tables;
queue_front_index += queue.size();
prev_ignored_logs = ignored_logs;
ignored_logs = 0;
exit_this_thread = is_shutdown;
return queue_front_index;
result.last_log_index = queue_front_index;
result.logs.swap(queue);
result.create_table_force = requested_prepare_tables > prepared_tables;
}
if (prev_ignored_logs)
LOG_ERROR(log, "Queue had been full at {}, accepted {} logs, ignored {} logs.",
result.last_log_index - result.logs.size(),
result.logs.size(),
prev_ignored_logs);
return result;
}
template <typename LogElement>
@ -229,13 +243,21 @@ SystemLogBase<LogElement>::SystemLogBase(
}
template <typename LogElement>
void SystemLogBase<LogElement>::flush(bool force)
SystemLogBase<LogElement>::Index SystemLogBase<LogElement>::getLastLogIndex()
{
uint64_t this_thread_requested_offset = queue->notifyFlush(force);
if (this_thread_requested_offset == uint64_t(-1))
return;
return queue->getLastLogIndex();
}
queue->waitFlush(this_thread_requested_offset);
template <typename LogElement>
void SystemLogBase<LogElement>::notifyFlush(Index expected_flushed_index, bool should_prepare_tables_anyway)
{
queue->notifyFlush(expected_flushed_index, should_prepare_tables_anyway);
}
template <typename LogElement>
void SystemLogBase<LogElement>::flush(Index expected_flushed_index, bool should_prepare_tables_anyway)
{
queue->waitFlush(expected_flushed_index, should_prepare_tables_anyway);
}
template <typename LogElement>
@ -257,9 +279,6 @@ void SystemLogBase<LogElement>::add(LogElement element)
queue->push(std::move(element));
}
template <typename LogElement>
void SystemLogBase<LogElement>::notifyFlush(bool force) { queue->notifyFlush(force); }
#define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogBase<ELEMENT>;
SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE)

View File

@ -1,6 +1,7 @@
#pragma once
#include <condition_variable>
#include <limits>
#include <memory>
#include <vector>
#include <base/types.h>
@ -54,10 +55,19 @@ struct StorageID;
class ISystemLog
{
public:
using Index = int64_t;
virtual String getName() const = 0;
//// force -- force table creation (used for SYSTEM FLUSH LOGS)
virtual void flush(bool force = false) = 0; /// NOLINT
/// Return the index of the latest added log element. That index no less than the flashed index.
/// The flashed index is the index of the last log element which has been flushed successfully.
/// Thereby all the records whose index is less than the flashed index are flushed already.
virtual Index getLastLogIndex() = 0;
/// Call this method to wake up the flush thread and flush the data in the background. It is non blocking call
virtual void notifyFlush(Index expected_flushed_index, bool should_prepare_tables_anyway) = 0;
/// Call this method to wait intill the logs are flushed up to expected_flushed_index. It is blocking call.
virtual void flush(Index expected_flushed_index, bool should_prepare_tables_anyway) = 0;
virtual void prepareTable() = 0;
/// Start the background thread.
@ -97,26 +107,38 @@ struct SystemLogQueueSettings
template <typename LogElement>
class SystemLogQueue
{
using Index = uint64_t;
public:
using Index = ISystemLog::Index;
explicit SystemLogQueue(const SystemLogQueueSettings & settings_);
void shutdown();
// producer methods
void push(LogElement && element);
Index notifyFlush(bool should_prepare_tables_anyway);
void waitFlush(Index expected_flushed_up_to);
Index getLastLogIndex();
void notifyFlush(Index expected_flushed_index, bool should_prepare_tables_anyway);
void waitFlush(Index expected_flushed_index, bool should_prepare_tables_anyway);
/// Handles crash, flushes log without blocking if notify_flush_on_crash is set
void handleCrash();
struct PopResult
{
Index last_log_index = 0;
std::vector<LogElement> logs = {};
bool create_table_force = false;
bool is_shutdown = false;
};
// consumer methods
Index pop(std::vector<LogElement>& output, bool & should_prepare_tables_anyway, bool & exit_this_thread);
void confirm(Index to_flush_end);
PopResult pop();
void confirm(Index last_flashed_index);
private:
void notifyFlushUnlocked(Index expected_flushed_index, bool should_prepare_tables_anyway);
/// Data shared between callers of add()/flush()/shutdown(), and the saving thread
std::mutex mutex;
@ -124,22 +146,32 @@ private:
// Queue is bounded. But its size is quite large to not block in all normal cases.
std::vector<LogElement> queue;
// An always-incrementing index of the first message currently in the queue.
// We use it to give a global sequential index to every message, so that we
// can wait until a particular message is flushed. This is used to implement
// synchronous log flushing for SYSTEM FLUSH LOGS.
Index queue_front_index = 0;
// A flag that says we must create the tables even if the queue is empty.
bool is_force_prepare_tables = false;
// Requested to flush logs up to this index, exclusive
Index requested_flush_up_to = 0;
Index requested_flush_index = std::numeric_limits<Index>::min();
// Flushed log up to this index, exclusive
Index flushed_up_to = 0;
// Logged overflow message at this queue front index
Index logged_queue_full_at_index = -1;
Index flushed_index = 0;
// The same logic for the prepare tables: if requested_prepar_tables > prepared_tables we need to do prepare
// except that initial prepared_tables is -1
// it is due to the difference: when no logs have been written and we call flush logs
// it becomes in the state: requested_flush_index = 0 and flushed_index = 0 -- we do not want to do anything
// but if we need to prepare tables it becomes requested_prepare_tables = 0 and prepared_tables = -1
// we trigger background thread and do prepare
Index requested_prepare_tables = std::numeric_limits<Index>::min();
Index prepared_tables = -1;
size_t ignored_logs = 0;
bool is_shutdown = false;
std::condition_variable confirm_event;
std::condition_variable flush_event;
const SystemLogQueueSettings settings;
@ -150,6 +182,7 @@ template <typename LogElement>
class SystemLogBase : public ISystemLog
{
public:
using Index = ISystemLog::Index;
using Self = SystemLogBase;
explicit SystemLogBase(
@ -163,15 +196,16 @@ public:
*/
void add(LogElement element);
Index getLastLogIndex() override;
void notifyFlush(Index expected_flushed_index, bool should_prepare_tables_anyway) override;
/// Flush data in the buffer to disk. Block the thread until the data is stored on disk.
void flush(bool force) override;
void flush(Index expected_flushed_index, bool should_prepare_tables_anyway) override;
/// Handles crash, flushes log without blocking if notify_flush_on_crash is set
void handleCrash() override;
/// Non-blocking flush data in the buffer to disk.
void notifyFlush(bool force);
String getName() const override { return LogElement::name(); }
static const char * getDefaultOrderBy() { return "event_date, event_time"; }

View File

@ -32,6 +32,8 @@
#cmakedefine01 USE_IDNA
#cmakedefine01 USE_NLP
#cmakedefine01 USE_VECTORSCAN
#cmakedefine01 USE_QPL
#cmakedefine01 USE_QATLIB
#cmakedefine01 USE_LIBURING
#cmakedefine01 USE_AVRO
#cmakedefine01 USE_CAPNP
@ -59,7 +61,7 @@
#cmakedefine01 USE_SKIM
#cmakedefine01 USE_PRQL
#cmakedefine01 USE_ULID
#cmakedefine01 FIU_ENABLE
#cmakedefine01 USE_LIBFIU
#cmakedefine01 USE_BCRYPT
#cmakedefine01 USE_LIBARCHIVE
#cmakedefine01 USE_POCKETFFT

View File

@ -174,19 +174,19 @@ struct Dictionary
{
switch (attribute.type)
{
case AttributeUnderlyingTypeTest::UInt8: std::get<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingTypeTest::UInt16: std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingTypeTest::UInt32: std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = static_cast<UInt32>(value.get<UInt64>()); break;
case AttributeUnderlyingTypeTest::UInt64: std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingTypeTest::Int8: std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingTypeTest::Int16: std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingTypeTest::Int32: std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = static_cast<Int32>(value.get<Int64>()); break;
case AttributeUnderlyingTypeTest::Int64: std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingTypeTest::Float32: std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = static_cast<Float32>(value.get<Float64>()); break;
case AttributeUnderlyingTypeTest::Float64: std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeUnderlyingTypeTest::UInt8: std::get<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = value.safeGet<UInt64>(); break;
case AttributeUnderlyingTypeTest::UInt16: std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = value.safeGet<UInt64>(); break;
case AttributeUnderlyingTypeTest::UInt32: std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = static_cast<UInt32>(value.safeGet<UInt64>()); break;
case AttributeUnderlyingTypeTest::UInt64: std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = value.safeGet<UInt64>(); break;
case AttributeUnderlyingTypeTest::Int8: std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = value.safeGet<Int64>(); break;
case AttributeUnderlyingTypeTest::Int16: std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = value.safeGet<Int64>(); break;
case AttributeUnderlyingTypeTest::Int32: std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = static_cast<Int32>(value.safeGet<Int64>()); break;
case AttributeUnderlyingTypeTest::Int64: std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = value.safeGet<Int64>(); break;
case AttributeUnderlyingTypeTest::Float32: std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = static_cast<Float32>(value.safeGet<Float64>()); break;
case AttributeUnderlyingTypeTest::Float64: std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = value.safeGet<Float64>(); break;
case AttributeUnderlyingTypeTest::String:
{
const auto & string = value.get<String>();
const auto & string = value.safeGet<String>();
auto & string_ref = std::get<ContainerPtrType<StringRef>>(attribute.arrays)[idx];
const auto & null_value_ref = std::get<String>(attribute.null_values);

View File

@ -1,7 +1,3 @@
#ifdef ENABLE_QPL_COMPRESSION
#include <cstdio>
#include <thread>
#include <Compression/CompressionCodecDeflateQpl.h>
#include <Compression/CompressionFactory.h>
#include <Compression/CompressionInfo.h>
@ -11,6 +7,10 @@
#include <Common/randomSeed.h>
#include <base/scope_guard.h>
#include <base/getPageSize.h>
#include <cstdio>
#include <thread>
#if USE_QPL
#include "libaccel_config.h"

View File

@ -4,6 +4,11 @@
#include <map>
#include <random>
#include <pcg_random.hpp>
#include "config.h"
#if USE_QPL
#include <qpl/qpl.h>
namespace Poco
@ -117,3 +122,4 @@ private:
};
}
#endif

View File

@ -1,4 +1,6 @@
#ifdef ENABLE_ZSTD_QAT_CODEC
#include "config.h"
#if USE_QATLIB
#include <Common/logger_useful.h>
#include <Compression/CompressionCodecZSTD.h>
@ -6,6 +8,7 @@
#include <Parsers/ASTLiteral.h>
#include <Parsers/IAST.h>
#include <Poco/Logger.h>
#include <qatseqprod.h>
#include <zstd.h>

View File

@ -1,20 +1,20 @@
#include "config.h"
#include <Compression/CompressionFactory.h>
#include <Compression/CompressionCodecMultiple.h>
#include <Compression/CompressionCodecNone.h>
#include <IO/ReadBuffer.h>
#include <IO/WriteHelpers.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Poco/String.h>
#include <IO/ReadBuffer.h>
#include <Parsers/queryToString.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Compression/CompressionCodecMultiple.h>
#include <Compression/CompressionCodecNone.h>
#include <IO/WriteHelpers.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <Poco/String.h>
#include <boost/algorithm/string/join.hpp>
#include "config.h"
namespace DB
{
@ -175,11 +175,11 @@ void registerCodecNone(CompressionCodecFactory & factory);
void registerCodecLZ4(CompressionCodecFactory & factory);
void registerCodecLZ4HC(CompressionCodecFactory & factory);
void registerCodecZSTD(CompressionCodecFactory & factory);
#ifdef ENABLE_ZSTD_QAT_CODEC
#if USE_QATLIB
void registerCodecZSTDQAT(CompressionCodecFactory & factory);
#endif
void registerCodecMultiple(CompressionCodecFactory & factory);
#ifdef ENABLE_QPL_COMPRESSION
#if USE_QPL
void registerCodecDeflateQpl(CompressionCodecFactory & factory);
#endif
@ -198,7 +198,7 @@ CompressionCodecFactory::CompressionCodecFactory()
registerCodecNone(*this);
registerCodecLZ4(*this);
registerCodecZSTD(*this);
#ifdef ENABLE_ZSTD_QAT_CODEC
#if USE_QATLIB
registerCodecZSTDQAT(*this);
#endif
registerCodecLZ4HC(*this);
@ -209,7 +209,7 @@ CompressionCodecFactory::CompressionCodecFactory()
registerCodecGorilla(*this);
registerCodecEncrypted(*this);
registerCodecFPC(*this);
#ifdef ENABLE_QPL_COMPRESSION
#if USE_QPL
registerCodecDeflateQpl(*this);
#endif
registerCodecGCD(*this);

View File

@ -49,7 +49,7 @@ ExternalTableDataPtr BaseExternalTable::getData(ContextPtr context)
{
initReadBuffer();
initSampleBlock();
auto input = context->getInputFormat(format, *read_buffer, sample_block, context->getSettingsRef().get("max_block_size").get<UInt64>());
auto input = context->getInputFormat(format, *read_buffer, sample_block, context->getSettingsRef().get("max_block_size").safeGet<UInt64>());
auto data = std::make_unique<ExternalTableData>();
data->pipe = std::make_unique<QueryPipelineBuilder>();

View File

@ -457,15 +457,6 @@ public:
std::string_view getTypeName() const;
bool isNull() const { return which == Types::Null; }
template <typename T>
NearestFieldType<std::decay_t<T>> & get();
template <typename T>
const auto & get() const
{
auto * mutable_this = const_cast<std::decay_t<decltype(*this)> *>(this);
return mutable_this->get<T>();
}
bool isNegativeInfinity() const { return which == Types::Null && get<Null>().isNegativeInfinity(); }
bool isPositiveInfinity() const { return which == Types::Null && get<Null>().isPositiveInfinity(); }
@ -681,6 +672,25 @@ private:
Types::Which which;
/// This function is prone to type punning and should never be used outside of Field class,
/// whenever it is used within this class the stored type should be checked in advance.
template <typename T>
NearestFieldType<std::decay_t<T>> & get()
{
// Before storing the value in the Field, we static_cast it to the field
// storage type, so here we return the value of storage type as well.
// Otherwise, it is easy to make a mistake of reinterpret_casting the stored
// value to a different and incompatible type.
// For example, a Float32 value is stored as Float64, and it is incorrect to
// return a reference to this value as Float32.
return *reinterpret_cast<NearestFieldType<std::decay_t<T>>*>(&storage);
}
template <typename T>
NearestFieldType<std::decay_t<T>> & get() const
{
return const_cast<Field *>(this)->get<T>();
}
/// Assuming there was no allocated state or it was deallocated (see destroy).
template <typename T>
@ -866,48 +876,21 @@ constexpr bool isInt64OrUInt64orBoolFieldType(Field::Types::Which t)
|| t == Field::Types::Bool;
}
// Field value getter with type checking in debug builds.
template <typename T>
NearestFieldType<std::decay_t<T>> & Field::get()
{
// Before storing the value in the Field, we static_cast it to the field
// storage type, so here we return the value of storage type as well.
// Otherwise, it is easy to make a mistake of reinterpret_casting the stored
// value to a different and incompatible type.
// For example, a Float32 value is stored as Float64, and it is incorrect to
// return a reference to this value as Float32.
using StoredType = NearestFieldType<std::decay_t<T>>;
#ifndef NDEBUG
// Disregard signedness when converting between int64 types.
constexpr Field::Types::Which target = TypeToEnum<StoredType>::value;
if (target != which
&& (!isInt64OrUInt64orBoolFieldType(target) || !isInt64OrUInt64orBoolFieldType(which)) && target != Field::Types::IPv4)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Invalid Field get from type {} to type {}", which, target);
#endif
StoredType * MAY_ALIAS ptr = reinterpret_cast<StoredType *>(&storage);
return *ptr;
}
template <typename T>
auto & Field::safeGet()
{
const Types::Which target = TypeToEnum<NearestFieldType<std::decay_t<T>>>::value;
/// We allow converting int64 <-> uint64, int64 <-> bool, uint64 <-> bool in safeGet().
if (target != which
&& (!isInt64OrUInt64orBoolFieldType(target) || !isInt64OrUInt64orBoolFieldType(which)))
throw Exception(ErrorCodes::BAD_GET,
"Bad get: has {}, requested {}", getTypeName(), target);
/// bool is stored as uint64, will be returned as UInt64 when requested as bool or UInt64, as Int64 when requested as Int64
/// also allow UInt64 <-> Int64 conversion
if (target != which &&
!(which == Field::Types::Bool && (target == Field::Types::UInt64 || target == Field::Types::Int64)) &&
!(isInt64OrUInt64FieldType(which) && isInt64OrUInt64FieldType(target)))
throw Exception(ErrorCodes::BAD_GET, "Bad get: has {}, requested {}", getTypeName(), target);
return get<T>();
}
template <typename T>
requires not_field_or_bool_or_stringlike<T>
Field::Field(T && rhs)

View File

@ -62,27 +62,27 @@ void Range::shrinkToIncludedIfPossible()
{
if (left.isExplicit() && !left_included)
{
if (left.getType() == Field::Types::UInt64 && left.get<UInt64>() != std::numeric_limits<UInt64>::max())
if (left.getType() == Field::Types::UInt64 && left.safeGet<UInt64>() != std::numeric_limits<UInt64>::max())
{
++left.get<UInt64 &>();
++left.safeGet<UInt64 &>();
left_included = true;
}
if (left.getType() == Field::Types::Int64 && left.get<Int64>() != std::numeric_limits<Int64>::max())
if (left.getType() == Field::Types::Int64 && left.safeGet<Int64>() != std::numeric_limits<Int64>::max())
{
++left.get<Int64 &>();
++left.safeGet<Int64 &>();
left_included = true;
}
}
if (right.isExplicit() && !right_included)
{
if (right.getType() == Field::Types::UInt64 && right.get<UInt64>() != std::numeric_limits<UInt64>::min())
if (right.getType() == Field::Types::UInt64 && right.safeGet<UInt64>() != std::numeric_limits<UInt64>::min())
{
--right.get<UInt64 &>();
--right.safeGet<UInt64 &>();
right_included = true;
}
if (right.getType() == Field::Types::Int64 && right.get<Int64>() != std::numeric_limits<Int64>::min())
if (right.getType() == Field::Types::Int64 && right.safeGet<Int64>() != std::numeric_limits<Int64>::min())
{
--right.get<Int64 &>();
--right.safeGet<Int64 &>();
right_included = true;
}
}

View File

@ -118,7 +118,7 @@ void Settings::set(std::string_view name, const Field & value)
{
if (value.getType() != Field::Types::Which::String)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type of value for setting 'compatibility'. Expected String, got {}", value.getTypeName());
applyCompatibilitySetting(value.get<String>());
applyCompatibilitySetting(value.safeGet<String>());
}
/// If we change setting that was changed by compatibility setting before
/// we should remove it from settings_changed_by_compatibility_setting,

View File

@ -53,29 +53,29 @@ namespace
{
if (f.getType() == Field::Types::String)
{
return stringToNumber<T>(f.get<const String &>());
return stringToNumber<T>(f.safeGet<const String &>());
}
else if (f.getType() == Field::Types::UInt64)
{
T result;
if (!accurate::convertNumeric(f.get<UInt64>(), result))
if (!accurate::convertNumeric(f.safeGet<UInt64>(), result))
throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Field value {} is out of range of {} type", f, demangle(typeid(T).name()));
return result;
}
else if (f.getType() == Field::Types::Int64)
{
T result;
if (!accurate::convertNumeric(f.get<Int64>(), result))
if (!accurate::convertNumeric(f.safeGet<Int64>(), result))
throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Field value {} is out of range of {} type", f, demangle(typeid(T).name()));
return result;
}
else if (f.getType() == Field::Types::Bool)
{
return T(f.get<bool>());
return T(f.safeGet<bool>());
}
else if (f.getType() == Field::Types::Float64)
{
Float64 x = f.get<Float64>();
Float64 x = f.safeGet<Float64>();
if constexpr (std::is_floating_point_v<T>)
{
return T(x);
@ -120,7 +120,7 @@ namespace
if (f.getType() == Field::Types::String)
{
/// Allow to parse Map from string field. For the convenience.
const auto & str = f.get<const String &>();
const auto & str = f.safeGet<const String &>();
return stringToMap(str);
}
@ -218,7 +218,7 @@ namespace
UInt64 fieldToMaxThreads(const Field & f)
{
if (f.getType() == Field::Types::String)
return stringToMaxThreads(f.get<const String &>());
return stringToMaxThreads(f.safeGet<const String &>());
else
return fieldToNumber<UInt64>(f);
}

View File

@ -100,7 +100,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log)
return current_value;
};
UInt64 max_threads = get_current_value("max_threads").get<UInt64>();
UInt64 max_threads = get_current_value("max_threads").safeGet<UInt64>();
UInt64 max_threads_max_value = 256 * getNumberOfPhysicalCPUCores();
if (max_threads > max_threads_max_value)
{
@ -120,7 +120,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log)
"input_format_parquet_max_block_size"};
for (auto const & setting : block_rows_settings)
{
if (auto block_size = get_current_value(setting).get<UInt64>();
if (auto block_size = get_current_value(setting).safeGet<UInt64>();
block_size > max_sane_block_rows_size)
{
if (log)
@ -129,7 +129,7 @@ void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log)
}
}
if (auto max_block_size = get_current_value("max_block_size").get<UInt64>(); max_block_size == 0)
if (auto max_block_size = get_current_value("max_block_size").safeGet<UInt64>(); max_block_size == 0)
{
if (log)
LOG_WARNING(log, "Sanity check: 'max_block_size' cannot be 0. Set to default value {}", DEFAULT_BLOCK_SIZE);

View File

@ -37,7 +37,7 @@ int main(int argc, char ** argv)
std::cerr << applyVisitor(to_string, field) << std::endl;
}
field.get<Array &>().push_back(field);
field.safeGet<Array &>().push_back(field);
std::cerr << applyVisitor(to_string, field) << std::endl;
std::cerr << (field < field2) << std::endl;

View File

@ -8,31 +8,31 @@ GTEST_TEST(Field, FromBool)
{
Field f{false};
ASSERT_EQ(f.getType(), Field::Types::Bool);
ASSERT_EQ(f.get<UInt64>(), 0);
ASSERT_EQ(f.get<bool>(), false);
ASSERT_EQ(f.safeGet<UInt64>(), 0);
ASSERT_EQ(f.safeGet<bool>(), false);
}
{
Field f{true};
ASSERT_EQ(f.getType(), Field::Types::Bool);
ASSERT_EQ(f.get<UInt64>(), 1);
ASSERT_EQ(f.get<bool>(), true);
ASSERT_EQ(f.safeGet<UInt64>(), 1);
ASSERT_EQ(f.safeGet<bool>(), true);
}
{
Field f;
f = false;
ASSERT_EQ(f.getType(), Field::Types::Bool);
ASSERT_EQ(f.get<UInt64>(), 0);
ASSERT_EQ(f.get<bool>(), false);
ASSERT_EQ(f.safeGet<UInt64>(), 0);
ASSERT_EQ(f.safeGet<bool>(), false);
}
{
Field f;
f = true;
ASSERT_EQ(f.getType(), Field::Types::Bool);
ASSERT_EQ(f.get<UInt64>(), 1);
ASSERT_EQ(f.get<bool>(), true);
ASSERT_EQ(f.safeGet<UInt64>(), 1);
ASSERT_EQ(f.safeGet<bool>(), true);
}
}
@ -42,15 +42,15 @@ GTEST_TEST(Field, Move)
Field f;
f = Field{String{"Hello, world (1)"}};
ASSERT_EQ(f.get<String>(), "Hello, world (1)");
ASSERT_EQ(f.safeGet<String>(), "Hello, world (1)");
f = Field{String{"Hello, world (2)"}};
ASSERT_EQ(f.get<String>(), "Hello, world (2)");
ASSERT_EQ(f.safeGet<String>(), "Hello, world (2)");
f = Field{Array{Field{String{"Hello, world (3)"}}}};
ASSERT_EQ(f.get<Array>()[0].get<String>(), "Hello, world (3)");
ASSERT_EQ(f.safeGet<Array>()[0].safeGet<String>(), "Hello, world (3)");
f = String{"Hello, world (4)"};
ASSERT_EQ(f.get<String>(), "Hello, world (4)");
ASSERT_EQ(f.safeGet<String>(), "Hello, world (4)");
f = Array{Field{String{"Hello, world (5)"}}};
ASSERT_EQ(f.get<Array>()[0].get<String>(), "Hello, world (5)");
ASSERT_EQ(f.safeGet<Array>()[0].safeGet<String>(), "Hello, world (5)");
f = Array{String{"Hello, world (6)"}};
ASSERT_EQ(f.get<Array>()[0].get<String>(), "Hello, world (6)");
ASSERT_EQ(f.safeGet<Array>()[0].safeGet<String>(), "Hello, world (6)");
}

View File

@ -129,7 +129,7 @@ MutableColumnPtr DataTypeAggregateFunction::createColumn() const
Field DataTypeAggregateFunction::getDefault() const
{
Field field = AggregateFunctionStateData();
field.get<AggregateFunctionStateData &>().name = getName();
field.safeGet<AggregateFunctionStateData &>().name = getName();
AlignedBuffer place_buffer(function->sizeOfData(), function->alignOfData());
AggregateDataPtr place = place_buffer.data();
@ -138,7 +138,7 @@ Field DataTypeAggregateFunction::getDefault() const
try
{
WriteBufferFromString buffer_from_field(field.get<AggregateFunctionStateData &>().data);
WriteBufferFromString buffer_from_field(field.safeGet<AggregateFunctionStateData &>().data);
function->serialize(place, buffer_from_field, version);
}
catch (...)

View File

@ -71,10 +71,10 @@ static DataTypePtr create(const ASTPtr & arguments)
auto * literal = argument->arguments->children[1]->as<ASTLiteral>();
if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.get<UInt64>() == 0 || literal->value.get<UInt64>() > ColumnVariant::MAX_NESTED_COLUMNS)
if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.safeGet<UInt64>() == 0 || literal->value.safeGet<UInt64>() > ColumnVariant::MAX_NESTED_COLUMNS)
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'max_types' argument for Dynamic type should be a positive integer between 1 and 255");
return std::make_shared<DataTypeDynamic>(literal->value.get<UInt64>());
return std::make_shared<DataTypeDynamic>(literal->value.safeGet<UInt64>());
}
void registerDataTypeDynamic(DataTypeFactory & factory)

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