Merge branch 'ClickHouse:master' into external_cross_join

This commit is contained in:
p1rattttt 2024-05-11 00:52:18 +03:00 committed by GitHub
commit ed43788062
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
118 changed files with 829 additions and 364 deletions

View File

@ -40,7 +40,6 @@ Every month we get together with the community (users, contributors, customers,
Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `<at>` clickhouse `<dot>` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc.
* [ClickHouse Meetup in Bengaluru](https://www.meetup.com/clickhouse-bangalore-user-group/events/300405581/) - May 4
* [ClickHouse Happy Hour @ Tom's Watch Bar - Los Angeles](https://www.meetup.com/clickhouse-los-angeles-user-group/events/300740584/) - May 22
* [ClickHouse & Confluent Meetup in Dubai](https://www.meetup.com/clickhouse-dubai-meetup-group/events/299629189/) - May 28
* [ClickHouse Meetup in Stockholm](https://www.meetup.com/clickhouse-stockholm-user-group/events/299752651/) - Jun 3
@ -49,6 +48,7 @@ Keep an eye out for upcoming meetups and events around the world. Somewhere else
* [ClickHouse Meetup in Amsterdam](https://www.meetup.com/clickhouse-netherlands-user-group/events/300781068/) - Jun 27
* [ClickHouse Meetup in Paris](https://www.meetup.com/clickhouse-france-user-group/events/300783448/) - Jul 9
* [ClickHouse Meetup @ Ramp - New York City](https://www.meetup.com/clickhouse-new-york-user-group/events/300595845/) - Jul 9
* [ClickHouse Meetup @ Klaviyo - Boston](https://www.meetup.com/clickhouse-boston-user-group/events/300907870) - Jul 11
## Recent Recordings
* **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments"

View File

@ -1,11 +1,14 @@
FROM ubuntu:20.04
# see https://github.com/moby/moby/issues/4032#issuecomment-192327844
# It could be removed after we move on a version 23:04+
ARG DEBIAN_FRONTEND=noninteractive
# ARG for quick switch to a given ubuntu mirror
ARG apt_archive="http://archive.ubuntu.com"
# We shouldn't use `apt upgrade` to not change the upstream image. It's updated biweekly
# user/group precreated explicitly with fixed uid/gid on purpose.
# It is especially important for rootless containers: in that case entrypoint
# can't do chown and owners of mounted volumes should be configured externally.
@ -16,13 +19,11 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list
&& groupadd -r clickhouse --gid=101 \
&& useradd -r -g clickhouse --uid=101 --home-dir=/var/lib/clickhouse --shell=/bin/bash clickhouse \
&& apt-get update \
&& apt-get upgrade -yq \
&& apt-get install --yes --no-install-recommends \
ca-certificates \
locales \
tzdata \
wget \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
ARG REPO_CHANNEL="stable"
@ -30,6 +31,9 @@ ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https
ARG VERSION="24.4.1.2088"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
#docker-official-library:off
# The part between `docker-official-library` tags is related to our builds
# set non-empty deb_location_url url to create a docker image
# from debs created by CI build, for example:
# docker build . --network host --build-arg version="21.4.1.6282" --build-arg deb_location_url="https://..." -t ...
@ -80,19 +84,22 @@ RUN if [ -n "${single_binary_location_url}" ]; then \
&& rm -rf /tmp/* ; \
fi
# The rest is the same in the official docker and in our build system
#docker-official-library:on
# A fallback to installation from ClickHouse repository
RUN if ! clickhouse local -q "SELECT ''" > /dev/null 2>&1; then \
apt-get update \
&& apt-get install --yes --no-install-recommends \
apt-transport-https \
ca-certificates \
dirmngr \
gnupg2 \
&& mkdir -p /etc/apt/sources.list.d \
&& GNUPGHOME=$(mktemp -d) \
&& GNUPGHOME="$GNUPGHOME" gpg --no-default-keyring \
&& GNUPGHOME="$GNUPGHOME" gpg --batch --no-default-keyring \
--keyring /usr/share/keyrings/clickhouse-keyring.gpg \
--keyserver hkp://keyserver.ubuntu.com:80 --recv-keys 8919F6BD2B48D754 \
--keyserver hkp://keyserver.ubuntu.com:80 \
--recv-keys 3a9ea1193a97b548be1457d48919f6bd2b48d754 \
&& rm -rf "$GNUPGHOME" \
&& chmod +r /usr/share/keyrings/clickhouse-keyring.gpg \
&& echo "${REPOSITORY}" > /etc/apt/sources.list.d/clickhouse.list \
@ -127,7 +134,6 @@ RUN mkdir /docker-entrypoint-initdb.d
COPY docker_related_config.xml /etc/clickhouse-server/config.d/
COPY entrypoint.sh /entrypoint.sh
RUN chmod +x /entrypoint.sh
EXPOSE 9000 8123 9009
VOLUME /var/lib/clickhouse

View File

@ -4,33 +4,34 @@
ClickHouse is an open-source column-oriented DBMS (columnar database management system) for online analytical processing (OLAP) that allows users to generate analytical reports using SQL queries in real-time.
ClickHouse works 100-1000x faster than traditional database management systems, and processes hundreds of millions to over a billion rows and tens of gigabytes of data per server per second. With a widespread user base around the globe, the technology has received praise for its reliability, ease of use, and fault tolerance.
ClickHouse works 100-1000x faster than traditional database management systems, and processes hundreds of millions to over a billion rows and tens of gigabytes of data per server per second. With a widespread user base around the globe, the technology has received praise for its reliability, ease of use, and fault tolerance.
For more information and documentation see https://clickhouse.com/.
## Versions
- The `latest` tag points to the latest release of the latest stable branch.
- Branch tags like `22.2` point to the latest release of the corresponding branch.
- Full version tags like `22.2.3.5` point to the corresponding release.
- The tag `head` is built from the latest commit to the default branch.
- Each tag has optional `-alpine` suffix to reflect that it's built on top of `alpine`.
- The `latest` tag points to the latest release of the latest stable branch.
- Branch tags like `22.2` point to the latest release of the corresponding branch.
- Full version tags like `22.2.3.5` point to the corresponding release.
- The tag `head` is built from the latest commit to the default branch.
- Each tag has optional `-alpine` suffix to reflect that it's built on top of `alpine`.
### Compatibility
- The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3.
- The arm64 image requires support for the [ARMv8.2-A architecture](https://en.wikipedia.org/wiki/AArch64#ARMv8.2-A) and additionally the Load-Acquire RCpc register. The register is optional in version ARMv8.2-A and mandatory in [ARMv8.3-A](https://en.wikipedia.org/wiki/AArch64#ARMv8.3-A). Supported in Graviton >=2, Azure and GCP instances. Examples for unsupported devices are Raspberry Pi 4 (ARMv8.0-A) and Jetson AGX Xavier/Orin (ARMv8.2-A).
- The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3.
- The arm64 image requires support for the [ARMv8.2-A architecture](https://en.wikipedia.org/wiki/AArch64#ARMv8.2-A) and additionally the Load-Acquire RCpc register. The register is optional in version ARMv8.2-A and mandatory in [ARMv8.3-A](https://en.wikipedia.org/wiki/AArch64#ARMv8.3-A). Supported in Graviton >=2, Azure and GCP instances. Examples for unsupported devices are Raspberry Pi 4 (ARMv8.0-A) and Jetson AGX Xavier/Orin (ARMv8.2-A).
## How to use this image
### start server instance
```bash
docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server
```
By default, ClickHouse will be accessible only via the Docker network. See the [networking section below](#networking).
By default, starting above server instance will be run as the `default` user without password.
By default, starting above server instance will be run as the `default` user without password.
### connect to it from a native client
@ -66,9 +67,7 @@ docker run -d -p 18123:8123 -p19000:9000 --name some-clickhouse-server --ulimit
echo 'SELECT version()' | curl 'http://localhost:18123/' --data-binary @-
```
```
22.6.3.35
```
`22.6.3.35`
or by allowing the container to use [host ports directly](https://docs.docker.com/network/host/) using `--network=host` (also allows achieving better network performance):
@ -77,16 +76,14 @@ docker run -d --network=host --name some-clickhouse-server --ulimit nofile=26214
echo 'SELECT version()' | curl 'http://localhost:8123/' --data-binary @-
```
```
22.6.3.35
```
`22.6.3.35`
### Volumes
Typically you may want to mount the following folders inside your container to achieve persistency:
* `/var/lib/clickhouse/` - main folder where ClickHouse stores the data
* `/var/log/clickhouse-server/` - logs
- `/var/lib/clickhouse/` - main folder where ClickHouse stores the data
- `/var/log/clickhouse-server/` - logs
```bash
docker run -d \
@ -97,9 +94,9 @@ docker run -d \
You may also want to mount:
* `/etc/clickhouse-server/config.d/*.xml` - files with server configuration adjustments
* `/etc/clickhouse-server/users.d/*.xml` - files with user settings adjustments
* `/docker-entrypoint-initdb.d/` - folder with database initialization scripts (see below).
- `/etc/clickhouse-server/config.d/*.xml` - files with server configuration adjustments
- `/etc/clickhouse-server/users.d/*.xml` - files with user settings adjustments
- `/docker-entrypoint-initdb.d/` - folder with database initialization scripts (see below).
### Linux capabilities
@ -150,7 +147,7 @@ docker run --rm -e CLICKHOUSE_DB=my_database -e CLICKHOUSE_USER=username -e CLIC
## How to extend this image
To perform additional initialization in an image derived from this one, add one or more `*.sql`, `*.sql.gz`, or `*.sh` scripts under `/docker-entrypoint-initdb.d`. After the entrypoint calls `initdb`, it will run any `*.sql` files, run any executable `*.sh` scripts, and source any non-executable `*.sh` scripts found in that directory to do further initialization before starting the service.
To perform additional initialization in an image derived from this one, add one or more `*.sql`, `*.sql.gz`, or `*.sh` scripts under `/docker-entrypoint-initdb.d`. After the entrypoint calls `initdb`, it will run any `*.sql` files, run any executable `*.sh` scripts, and source any non-executable `*.sh` scripts found in that directory to do further initialization before starting the service.
Also, you can provide environment variables `CLICKHOUSE_USER` & `CLICKHOUSE_PASSWORD` that will be used for clickhouse-client during initialization.
For example, to add an additional user and database, add the following to `/docker-entrypoint-initdb.d/init-db.sh`:

View File

@ -51,6 +51,9 @@ ENGINE = MaterializedMySQL('host:port', ['database' | database], 'user', 'passwo
### allows_query_when_mysql_lost
`allows_query_when_mysql_lost` — Allows to query a materialized table when MySQL is lost. Default: `0` (`false`).
### allow_startup_database_without_connection_to_mysql
`allow_startup_database_without_connection_to_mysql` — Allow to create and attach database without available connection to MySQL. Default: `0` (`false`).
### materialized_mysql_tables_list
`materialized_mysql_tables_list` — a comma-separated list of mysql database tables, which will be replicated by MaterializedMySQL database engine. Default value: empty list — means whole tables will be replicated.

View File

@ -10,7 +10,7 @@ sidebar_label: Data Replication
In ClickHouse Cloud replication is managed for you. Please create your tables without adding arguments. For example, in the text below you would replace:
```sql
ENGINE = ReplicatedReplacingMergeTree(
ENGINE = ReplicatedMergeTree(
'/clickhouse/tables/{shard}/table_name',
'{replica}',
ver
@ -20,7 +20,7 @@ ENGINE = ReplicatedReplacingMergeTree(
with:
```sql
ENGINE = ReplicatedReplacingMergeTree
ENGINE = ReplicatedMergeTree
```
:::
@ -140,11 +140,11 @@ The system monitors data synchronicity on replicas and is able to recover after
:::note
In ClickHouse Cloud replication is managed for you. Please create your tables without adding arguments. For example, in the text below you would replace:
```
ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{shard}/table_name', '{replica}', ver)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/table_name', '{replica}', ver)
```
with:
```
ENGINE = ReplicatedReplacingMergeTree
ENGINE = ReplicatedMergeTree
```
:::
@ -177,7 +177,7 @@ CREATE TABLE table_name
CounterID UInt32,
UserID UInt32,
ver UInt16
) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{shard}/table_name', '{replica}', ver)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/table_name', '{replica}', ver)
PARTITION BY toYYYYMM(EventDate)
ORDER BY (CounterID, EventDate, intHash32(UserID))
SAMPLE BY intHash32(UserID);

View File

@ -1058,7 +1058,7 @@ convertCharset(s, from, to)
## base58Encode
Encodes a String using [Base58](https://tools.ietf.org/id/draft-msporny-base58-01.html) in the "Bitcoin" alphabet.
Encodes a String using [Base58](https://datatracker.ietf.org/doc/html/draft-msporny-base58) in the "Bitcoin" alphabet.
**Syntax**
@ -1092,7 +1092,7 @@ Result:
## base58Decode
Accepts a String and decodes it using [Base58](https://tools.ietf.org/id/draft-msporny-base58-01.html) encoding scheme using "Bitcoin" alphabet.
Accepts a String and decodes it using [Base58](https://datatracker.ietf.org/doc/html/draft-msporny-base58) encoding scheme using "Bitcoin" alphabet.
**Syntax**

View File

@ -493,7 +493,7 @@ SELECT concat(key1, key2), sum(value) FROM key_val GROUP BY (key1, key2);
## base58Encode(plaintext), base58Decode(encoded_text) {#base58}
Принимает на вход строку или колонку строк и кодирует/раскодирует их с помощью схемы кодирования [Base58](https://tools.ietf.org/id/draft-msporny-base58-01.html) с использованием стандартного алфавита Bitcoin.
Принимает на вход строку или колонку строк и кодирует/раскодирует их с помощью схемы кодирования [Base58](https://datatracker.ietf.org/doc/html/draft-msporny-base58) с использованием стандартного алфавита Bitcoin.
**Синтаксис**

View File

@ -538,9 +538,57 @@ let params = default_params;
/// Palette generation for charts
function generatePalette(numColors) {
// oklch() does not work in firefox<=125 inside <canvas> element so we convert it back to rgb for now.
// Based on https://github.com/color-js/color.js/blob/main/src/spaces/oklch.js
const multiplyMatrices = (A, B) => {
return [
A[0]*B[0] + A[1]*B[1] + A[2]*B[2],
A[3]*B[0] + A[4]*B[1] + A[5]*B[2],
A[6]*B[0] + A[7]*B[1] + A[8]*B[2]
];
}
const oklch2oklab = ([l, c, h]) => [
l,
isNaN(h) ? 0 : c * Math.cos(h * Math.PI / 180),
isNaN(h) ? 0 : c * Math.sin(h * Math.PI / 180)
]
const srgbLinear2rgb = rgb => rgb.map(c =>
Math.abs(c) > 0.0031308 ?
(c < 0 ? -1 : 1) * (1.055 * (Math.abs(c) ** (1 / 2.4)) - 0.055) :
12.92 * c
)
const oklab2xyz = lab => {
const LMSg = multiplyMatrices([
1, 0.3963377773761749, 0.2158037573099136,
1, -0.1055613458156586, -0.0638541728258133,
1, -0.0894841775298119, -1.2914855480194092,
], lab)
const LMS = LMSg.map(val => val ** 3)
return multiplyMatrices([
1.2268798758459243, -0.5578149944602171, 0.2813910456659647,
-0.0405757452148008, 1.1122868032803170, -0.0717110580655164,
-0.0763729366746601, -0.4214933324022432, 1.5869240198367816
], LMS)
}
const xyz2rgbLinear = xyz => {
return multiplyMatrices([
3.2409699419045226, -1.537383177570094, -0.4986107602930034,
-0.9692436362808796, 1.8759675015077202, 0.04155505740717559,
0.05563007969699366, -0.20397695888897652, 1.0569715142428786
], xyz)
}
const oklch2rgb = lch => srgbLinear2rgb(xyz2rgbLinear(oklab2xyz(oklch2oklab(lch))))
palette = [];
for (let i = 0; i < numColors; i++) {
palette.push(`oklch(${theme != 'dark' ? 0.75 : 0.5}, 0.15, ${360 * i / numColors})`);
//palette.push(`oklch(${theme != 'dark' ? 0.75 : 0.5}, 0.15, ${360 * i / numColors})`);
let rgb = oklch2rgb([theme != 'dark' ? 0.75 : 0.5, 0.15, 360 * i / numColors]);
palette.push(`rgb(${rgb[0] * 255}, ${rgb[1] * 255}, ${rgb[2] * 255})`);
}
return palette;
}

View File

@ -711,7 +711,7 @@ auto parseArguments(const std::string & name, const DataTypes & arguments)
const auto * array_type = checkAndGetDataType<DataTypeArray>(args[0].get());
if (!array_type)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an array, not {}",
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument #1 for function {} must be an array, not {}",
name, args[0]->getName());
DataTypePtr keys_type = array_type->getNestedType();
@ -722,8 +722,8 @@ auto parseArguments(const std::string & name, const DataTypes & arguments)
{
array_type = checkAndGetDataType<DataTypeArray>(args[i].get());
if (!array_type)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument #{} for function {} must be an array.",
i, name);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument #{} for function {} must be an array, not {}",
i + 1, name, args[i]->getName());
values_types.push_back(array_type->getNestedType());
}

View File

@ -1283,7 +1283,7 @@ ColumnPtr ColumnArray::replicateTuple(const Offsets & replicate_offsets) const
size_t ColumnArray::getNumberOfDimensions() const
{
const auto * nested_array = checkAndGetColumn<ColumnArray>(*data);
const auto * nested_array = checkAndGetColumn<ColumnArray>(&*data);
if (!nested_array)
return 1;
return 1 + nested_array->getNumberOfDimensions(); /// Every modern C++ compiler optimizes tail recursion.

View File

@ -903,7 +903,7 @@ ColumnPtr ColumnLowCardinality::cloneWithDefaultOnNull() const
bool isColumnLowCardinalityNullable(const IColumn & column)
{
if (const auto * lc_column = checkAndGetColumn<ColumnLowCardinality>(column))
if (const auto * lc_column = checkAndGetColumn<ColumnLowCardinality>(&column))
return lc_column->nestedIsNullable();
return false;
}

View File

@ -164,7 +164,7 @@ bool ColumnTuple::tryInsert(const Field & x)
if (!columns[i]->tryInsert(tuple[i]))
{
for (size_t j = 0; j != i; ++j)
columns[i]->popBack(1);
columns[j]->popBack(1);
return false;
}

View File

@ -376,7 +376,7 @@ size_t ColumnUnique<ColumnType>::uniqueInsertFrom(const IColumn & src, size_t n)
if (is_nullable && src.isNullAt(n))
return getNullValueIndex();
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(src))
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(&src))
return uniqueInsertFrom(nullable->getNestedColumn(), n);
auto ref = src.getDataAt(n);
@ -569,7 +569,7 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
return nullptr;
};
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(src))
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(&src))
{
src_column = typeid_cast<const ColumnType *>(&nullable_column->getNestedColumn());
null_map = &nullable_column->getNullMapData();

View File

@ -32,7 +32,7 @@ ConstantFilterDescription::ConstantFilterDescription(const IColumn & column)
if (!typeid_cast<const ColumnUInt8 *>(column_nested.get()))
{
const ColumnNullable * column_nested_nullable = checkAndGetColumn<ColumnNullable>(*column_nested);
const ColumnNullable * column_nested_nullable = checkAndGetColumn<ColumnNullable>(&*column_nested);
if (!column_nested_nullable || !typeid_cast<const ColumnUInt8 *>(&column_nested_nullable->getNestedColumn()))
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER,
@ -66,7 +66,7 @@ FilterDescription::FilterDescription(const IColumn & column_)
return;
}
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(column))
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(&column))
{
ColumnPtr nested_column = nullable_column->getNestedColumnPtr();
MutableColumnPtr mutable_holder = IColumn::mutate(std::move(nested_column));

View File

@ -640,12 +640,16 @@ template <>
struct IsMutableColumns<> { static const bool value = true; };
/// Throws LOGICAL_ERROR if the type doesn't match.
template <typename Type>
const Type * checkAndGetColumn(const IColumn & column)
const Type & checkAndGetColumn(const IColumn & column)
{
return typeid_cast<const Type *>(&column);
return typeid_cast<const Type &>(column);
}
/// Returns nullptr if the type doesn't match.
/// If you're going to dereference the returned pointer without checking for null, use the
/// `const IColumn &` overload above instead.
template <typename Type>
const Type * checkAndGetColumn(const IColumn * column)
{

View File

@ -205,10 +205,10 @@ static MaskInfo extractMaskImpl(
auto column = col->convertToFullColumnIfLowCardinality();
/// Special implementation for Null and Const columns.
if (column->onlyNull() || checkAndGetColumn<ColumnConst>(*column))
if (column->onlyNull() || checkAndGetColumn<ColumnConst>(&*column))
return extractMaskFromConstOrNull<inverted>(mask, column, null_value, nulls);
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(*column))
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(&*column))
{
const PaddedPODArray<UInt8> & null_map = nullable_column->getNullMapData();
return extractMaskImpl<inverted>(mask, nullable_column->getNestedColumnPtr(), null_value, &null_map, nulls);

View File

@ -44,8 +44,8 @@ struct HashMethodOneNumber
{
if constexpr (nullable)
{
const auto * null_column = checkAndGetColumn<ColumnNullable>(key_columns[0]);
vec = null_column->getNestedColumnPtr()->getRawData().data();
const auto & null_column = checkAndGetColumn<ColumnNullable>(*key_columns[0]);
vec = null_column.getNestedColumnPtr()->getRawData().data();
}
else
{
@ -57,8 +57,8 @@ struct HashMethodOneNumber
{
if constexpr (nullable)
{
const auto * null_column = checkAndGetColumn<ColumnNullable>(column);
vec = null_column->getNestedColumnPtr()->getRawData().data();
const auto & null_column = checkAndGetColumn<ColumnNullable>(*column);
vec = null_column.getNestedColumnPtr()->getRawData().data();
}
else
{
@ -105,7 +105,7 @@ struct HashMethodString
const IColumn * column;
if constexpr (nullable)
{
column = checkAndGetColumn<ColumnNullable>(key_columns[0])->getNestedColumnPtr().get();
column = checkAndGetColumn<ColumnNullable>(*key_columns[0]).getNestedColumnPtr().get();
}
else
{
@ -153,7 +153,7 @@ struct HashMethodFixedString
const IColumn * column;
if constexpr (nullable)
{
column = checkAndGetColumn<ColumnNullable>(key_columns[0])->getNestedColumnPtr().get();
column = checkAndGetColumn<ColumnNullable>(*key_columns[0]).getNestedColumnPtr().get();
}
else
{

View File

@ -305,7 +305,7 @@ protected:
}
if constexpr (nullable)
null_map = &checkAndGetColumn<ColumnNullable>(column)->getNullMapColumn();
null_map = &checkAndGetColumn<ColumnNullable>(*column).getNullMapColumn();
}
template <typename Data, typename KeyHolder>

View File

@ -170,11 +170,11 @@ private:
if (c0_is_const && c1_is_const)
{
const ColumnConst * c0_const = checkAndGetColumnConst<ColVecA>(c0.get());
const ColumnConst * c1_const = checkAndGetColumnConst<ColVecB>(c1.get());
const ColumnConst & c0_const = checkAndGetColumnConst<ColVecA>(*c0);
const ColumnConst & c1_const = checkAndGetColumnConst<ColVecB>(*c1);
A a = c0_const->template getValue<A>();
B b = c1_const->template getValue<B>();
A a = c0_const.template getValue<A>();
B b = c1_const.template getValue<B>();
UInt8 res = apply<scale_left, scale_right>(a, b, scale);
return DataTypeUInt8().createColumnConst(c0->size(), toField(res));
}
@ -184,8 +184,8 @@ private:
if (c0_is_const)
{
const ColumnConst * c0_const = checkAndGetColumnConst<ColVecA>(c0.get());
A a = c0_const->template getValue<A>();
const ColumnConst & c0_const = checkAndGetColumnConst<ColVecA>(*c0);
A a = c0_const.template getValue<A>();
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
constantVector<scale_left, scale_right>(a, c1_vec->getData(), vec_res, scale);
else
@ -193,8 +193,8 @@ private:
}
else if (c1_is_const)
{
const ColumnConst * c1_const = checkAndGetColumnConst<ColVecB>(c1.get());
B b = c1_const->template getValue<B>();
const ColumnConst & c1_const = checkAndGetColumnConst<ColVecB>(*c1);
B b = c1_const.template getValue<B>();
if (const ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get()))
vectorConstant<scale_left, scale_right>(c0_vec->getData(), b, vec_res, scale);
else

View File

@ -47,7 +47,7 @@ size_t getNumberOfDimensions(const IDataType & type)
size_t getNumberOfDimensions(const IColumn & column)
{
if (const auto * column_array = checkAndGetColumn<ColumnArray>(column))
if (const auto * column_array = checkAndGetColumn<ColumnArray>(&column))
return column_array->getNumberOfDimensions();
return 0;
}

View File

@ -28,7 +28,7 @@ constexpr char str_false[6] = "false";
const ColumnUInt8 * checkAndGetSerializeColumnType(const IColumn & column)
{
const auto * col = checkAndGetColumn<ColumnUInt8>(&column);
if (!checkAndGetColumn<ColumnUInt8>(&column))
if (!col)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Bool type can only serialize columns of type UInt8.{}", column.getName());
return col;
}
@ -36,7 +36,7 @@ const ColumnUInt8 * checkAndGetSerializeColumnType(const IColumn & column)
ColumnUInt8 * checkAndGetDeserializeColumnType(IColumn & column)
{
auto * col = typeid_cast<ColumnUInt8 *>(&column);
if (!checkAndGetColumn<ColumnUInt8>(&column))
if (!col)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Bool type can only deserialize columns of type UInt8.{}",
column.getName());
return col;

View File

@ -17,7 +17,7 @@ namespace ErrorCodes
void SerializationKustoInterval::serializeText(
const IColumn & column, const size_t row, WriteBuffer & ostr, const FormatSettings &) const
{
const auto * interval_column = checkAndGetColumn<ColumnInterval>(column);
const auto * interval_column = checkAndGetColumn<ColumnInterval>(&column);
if (!interval_column)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval");

View File

@ -477,7 +477,7 @@ void SerializationLowCardinality::serializeBinaryBulkWithMultipleStreams(
settings.low_cardinality_max_dictionary_size);
}
if (const auto * nullable_keys = checkAndGetColumn<ColumnNullable>(*keys))
if (const auto * nullable_keys = checkAndGetColumn<ColumnNullable>(&*keys))
keys = nullable_keys->getNestedColumnPtr();
bool need_additional_keys = !keys->empty();

View File

@ -84,7 +84,8 @@ LoadTaskPtr DatabaseMaterializedMySQL::startupDatabaseAsync(AsyncLoader & async_
[this, mode] (AsyncLoader &, const LoadJobPtr &)
{
LOG_TRACE(log, "Starting MaterializeMySQL database");
if (mode < LoadingStrictnessLevel::FORCE_ATTACH)
if (!settings->allow_startup_database_without_connection_to_mysql
&& mode < LoadingStrictnessLevel::FORCE_ATTACH)
materialize_thread.assertMySQLAvailable();
materialize_thread.startSynchronization();

View File

@ -22,6 +22,7 @@ class ASTStorage;
M(UInt64, max_milliseconds_to_wait_in_binlog_queue, 10000, "Max milliseconds to wait when max bytes exceeded in a binlog queue.", 0) \
M(UInt64, max_bytes_in_binlog_dispatcher_buffer, DBMS_DEFAULT_BUFFER_SIZE, "Max bytes in the binlog dispatcher's buffer before it is flushed to attached binlogs.", 0) \
M(UInt64, max_flush_milliseconds_in_binlog_dispatcher, 1000, "Max milliseconds in the binlog dispatcher's buffer to wait before it is flushed to attached binlogs.", 0) \
M(Bool, allow_startup_database_without_connection_to_mysql, false, "Allow to create and attach database without available connection to MySQL.", 0) \
DECLARE_SETTINGS_TRAITS(MaterializedMySQLSettingsTraits, LIST_OF_MATERIALIZE_MODE_SETTINGS)

View File

@ -95,7 +95,7 @@ namespace
parent_key_column_non_null = parent_key_column_typed->getNestedColumnPtr();
}
const auto * parent_key_column_typed = checkAndGetColumn<ColumnVector<UInt64>>(*parent_key_column_non_null);
const auto * parent_key_column_typed = checkAndGetColumn<ColumnVector<UInt64>>(&*parent_key_column_non_null);
if (!parent_key_column_typed)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Parent key column should be UInt64. Actual {}",
@ -166,7 +166,7 @@ ColumnPtr getKeysHierarchyDefaultImplementation(
valid_keys = 0;
key_column = key_column->convertToFullColumnIfConst();
const auto * key_column_typed = checkAndGetColumn<ColumnVector<UInt64>>(*key_column);
const auto * key_column_typed = checkAndGetColumn<ColumnVector<UInt64>>(&*key_column);
if (!key_column_typed)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64");
@ -224,11 +224,11 @@ ColumnUInt8::Ptr getKeysIsInHierarchyDefaultImplementation(
key_column = key_column->convertToFullColumnIfConst();
in_key_column = in_key_column->convertToFullColumnIfConst();
const auto * key_column_typed = checkAndGetColumn<ColumnVector<UInt64>>(*key_column);
const auto * key_column_typed = checkAndGetColumn<ColumnVector<UInt64>>(&*key_column);
if (!key_column_typed)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64");
const auto * in_key_column_typed = checkAndGetColumn<ColumnVector<UInt64>>(*in_key_column);
const auto * in_key_column_typed = checkAndGetColumn<ColumnVector<UInt64>>(&*in_key_column);
if (!in_key_column_typed)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64");

View File

@ -19,45 +19,45 @@ bool insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const Col
if (isArray(input_column.type) && isArray(header_column.type))
{
ColumnWithTypeAndName nested_input_column;
const auto * array_input_column = checkAndGetColumn<ColumnArray>(input_column.column.get());
nested_input_column.column = array_input_column->getDataPtr();
const auto & array_input_column = checkAndGetColumn<ColumnArray>(*input_column.column);
nested_input_column.column = array_input_column.getDataPtr();
nested_input_column.type = checkAndGetDataType<DataTypeArray>(input_column.type.get())->getNestedType();
ColumnWithTypeAndName nested_header_column;
nested_header_column.column = checkAndGetColumn<ColumnArray>(header_column.column.get())->getDataPtr();
nested_header_column.column = checkAndGetColumn<ColumnArray>(*header_column.column).getDataPtr();
nested_header_column.type = checkAndGetDataType<DataTypeArray>(header_column.type.get())->getNestedType();
if (!insertNullAsDefaultIfNeeded(nested_input_column, nested_header_column, 0, nullptr))
return false;
input_column.column = ColumnArray::create(nested_input_column.column, array_input_column->getOffsetsPtr());
input_column.column = ColumnArray::create(nested_input_column.column, array_input_column.getOffsetsPtr());
input_column.type = std::make_shared<DataTypeArray>(std::move(nested_input_column.type));
return true;
}
if (isTuple(input_column.type) && isTuple(header_column.type))
{
const auto * tuple_input_column = checkAndGetColumn<ColumnTuple>(input_column.column.get());
const auto * tuple_input_type = checkAndGetDataType<DataTypeTuple>(input_column.type.get());
const auto * tuple_header_column = checkAndGetColumn<ColumnTuple>(header_column.column.get());
const auto * tuple_header_type = checkAndGetDataType<DataTypeTuple>(header_column.type.get());
const auto & tuple_input_column = checkAndGetColumn<ColumnTuple>(*input_column.column);
const auto & tuple_input_type = checkAndGetDataType<DataTypeTuple>(*input_column.type);
const auto & tuple_header_column = checkAndGetColumn<ColumnTuple>(*header_column.column);
const auto & tuple_header_type = checkAndGetDataType<DataTypeTuple>(*header_column.type);
if (tuple_input_type->getElements().size() != tuple_header_type->getElements().size())
if (tuple_input_type.getElements().size() != tuple_header_type.getElements().size())
return false;
Columns nested_input_columns;
nested_input_columns.reserve(tuple_input_type->getElements().size());
nested_input_columns.reserve(tuple_input_type.getElements().size());
DataTypes nested_input_types;
nested_input_types.reserve(tuple_input_type->getElements().size());
nested_input_types.reserve(tuple_input_type.getElements().size());
bool changed = false;
for (size_t i = 0; i != tuple_input_type->getElements().size(); ++i)
for (size_t i = 0; i != tuple_input_type.getElements().size(); ++i)
{
ColumnWithTypeAndName nested_input_column;
nested_input_column.column = tuple_input_column->getColumnPtr(i);
nested_input_column.type = tuple_input_type->getElement(i);
nested_input_column.column = tuple_input_column.getColumnPtr(i);
nested_input_column.type = tuple_input_type.getElement(i);
ColumnWithTypeAndName nested_header_column;
nested_header_column.column = tuple_header_column->getColumnPtr(i);
nested_header_column.type = tuple_header_type->getElement(i);
nested_header_column.column = tuple_header_column.getColumnPtr(i);
nested_header_column.type = tuple_header_type.getElement(i);
changed |= insertNullAsDefaultIfNeeded(nested_input_column, nested_header_column, 0, nullptr);
nested_input_columns.push_back(std::move(nested_input_column.column));
nested_input_types.push_back(std::move(nested_input_column.type));
@ -74,12 +74,12 @@ bool insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const Col
if (isMap(input_column.type) && isMap(header_column.type))
{
ColumnWithTypeAndName nested_input_column;
nested_input_column.column = checkAndGetColumn<ColumnMap>(input_column.column.get())->getNestedColumnPtr();
nested_input_column.type = checkAndGetDataType<DataTypeMap>(input_column.type.get())->getNestedType();
nested_input_column.column = checkAndGetColumn<ColumnMap>(*input_column.column).getNestedColumnPtr();
nested_input_column.type = checkAndGetDataType<DataTypeMap>(*input_column.type).getNestedType();
ColumnWithTypeAndName nested_header_column;
nested_header_column.column = checkAndGetColumn<ColumnMap>(header_column.column.get())->getNestedColumnPtr();
nested_header_column.type = checkAndGetDataType<DataTypeMap>(header_column.type.get())->getNestedType();
nested_header_column.column = checkAndGetColumn<ColumnMap>(*header_column.column).getNestedColumnPtr();
nested_header_column.type = checkAndGetDataType<DataTypeMap>(*header_column.type).getNestedType();
if (!insertNullAsDefaultIfNeeded(nested_input_column, nested_header_column, 0, nullptr))
return false;

View File

@ -111,9 +111,9 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const auto & input_column = arguments[0].column;
if (const auto * src_column_as_fixed_string = checkAndGetColumn<ColumnFixedString>(*input_column))
if (const auto * src_column_as_fixed_string = checkAndGetColumn<ColumnFixedString>(&*input_column))
return execute(*src_column_as_fixed_string, input_rows_count);
else if (const auto * src_column_as_string = checkAndGetColumn<ColumnString>(*input_column))
else if (const auto * src_column_as_string = checkAndGetColumn<ColumnString>(&*input_column))
return execute(*src_column_as_string, input_rows_count);
throw Exception(

View File

@ -1767,8 +1767,8 @@ public:
{
if (const auto * col_right_const = checkAndGetColumnConst<ColumnFixedString>(col_right_raw))
{
const auto * col_left = checkAndGetColumn<ColumnFixedString>(col_left_const->getDataColumn());
const auto * col_right = checkAndGetColumn<ColumnFixedString>(col_right_const->getDataColumn());
const auto * col_left = &checkAndGetColumn<ColumnFixedString>(col_left_const->getDataColumn());
const auto * col_right = &checkAndGetColumn<ColumnFixedString>(col_right_const->getDataColumn());
if (col_left->getN() != col_right->getN())
return nullptr;
@ -1805,11 +1805,11 @@ public:
const auto * col_left = is_left_column_const
? checkAndGetColumn<ColumnFixedString>(
checkAndGetColumnConst<ColumnFixedString>(col_left_raw)->getDataColumn())
&checkAndGetColumnConst<ColumnFixedString>(col_left_raw)->getDataColumn())
: checkAndGetColumn<ColumnFixedString>(col_left_raw);
const auto * col_right = is_right_column_const
? checkAndGetColumn<ColumnFixedString>(
checkAndGetColumnConst<ColumnFixedString>(col_right_raw)->getDataColumn())
&checkAndGetColumnConst<ColumnFixedString>(col_right_raw)->getDataColumn())
: checkAndGetColumn<ColumnFixedString>(col_right_raw);
if (col_left && col_right)
@ -1881,8 +1881,8 @@ public:
{
if (const auto * col_right_const = checkAndGetColumnConst<ColumnString>(col_right_raw))
{
const auto * col_left = checkAndGetColumn<ColumnString>(col_left_const->getDataColumn());
const auto * col_right = checkAndGetColumn<ColumnString>(col_right_const->getDataColumn());
const auto * col_left = &checkAndGetColumn<ColumnString>(col_left_const->getDataColumn());
const auto * col_right = &checkAndGetColumn<ColumnString>(col_right_const->getDataColumn());
std::string_view a = col_left->getDataAt(0).toView();
std::string_view b = col_right->getDataAt(0).toView();
@ -1897,10 +1897,10 @@ public:
const bool is_right_column_const = checkAndGetColumnConst<ColumnString>(col_right_raw) != nullptr;
const auto * col_left = is_left_column_const
? checkAndGetColumn<ColumnString>(checkAndGetColumnConst<ColumnString>(col_left_raw)->getDataColumn())
? &checkAndGetColumn<ColumnString>(checkAndGetColumnConst<ColumnString>(col_left_raw)->getDataColumn())
: checkAndGetColumn<ColumnString>(col_left_raw);
const auto * col_right = is_right_column_const
? checkAndGetColumn<ColumnString>(checkAndGetColumnConst<ColumnString>(col_right_raw)->getDataColumn())
? &checkAndGetColumn<ColumnString>(checkAndGetColumnConst<ColumnString>(col_right_raw)->getDataColumn())
: checkAndGetColumn<ColumnString>(col_right_raw);
if (col_left && col_right)
@ -1948,7 +1948,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A
const ColumnConst * const col_left_const = checkAndGetColumnConst<LeftColumnType>(col_left_raw);
const auto * col_left = col_left_const ? checkAndGetColumn<LeftColumnType>(col_left_const->getDataColumn())
const auto * col_left = col_left_const ? &checkAndGetColumn<LeftColumnType>(col_left_const->getDataColumn())
: checkAndGetColumn<LeftColumnType>(col_left_raw);
if (!col_left)
@ -2231,7 +2231,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A
bool is_const = checkColumnConst<ColumnNullable>(right_argument.column.get());
const ColumnNullable * nullable_column = is_const ? checkAndGetColumnConstData<ColumnNullable>(right_argument.column.get())
: checkAndGetColumn<ColumnNullable>(*right_argument.column);
: checkAndGetColumn<ColumnNullable>(right_argument.column.get());
const auto & null_bytemap = nullable_column->getNullMapData();
auto res = executeImpl2(createBlockWithNestedColumns(arguments), removeNullable(result_type), input_rows_count, &null_bytemap);

View File

@ -58,14 +58,14 @@ ColumnWithTypeAndName columnGetNested(const ColumnWithTypeAndName & col)
{
return ColumnWithTypeAndName{nullptr, nested_type, col.name};
}
else if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*col.column))
else if (const auto * nullable = checkAndGetColumn<ColumnNullable>(&*col.column))
{
const auto & nested_col = nullable->getNestedColumnPtr();
return ColumnWithTypeAndName{nested_col, nested_type, col.name};
}
else if (const auto * const_column = checkAndGetColumn<ColumnConst>(*col.column))
else if (const auto * const_column = checkAndGetColumn<ColumnConst>(&*col.column))
{
const auto * nullable_column = checkAndGetColumn<ColumnNullable>(const_column->getDataColumn());
const auto * nullable_column = checkAndGetColumn<ColumnNullable>(&const_column->getDataColumn());
ColumnPtr nullable_res;
if (nullable_column)
@ -226,7 +226,7 @@ ColumnPtr wrapInNullable(const ColumnPtr & src, const ColumnsWithTypeAndName & a
if (src->onlyNull())
return src;
else if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*src))
else if (const auto * nullable = checkAndGetColumn<ColumnNullable>(&*src))
{
src_not_nullable = nullable->getNestedColumnPtr();
result_null_map_column = nullable->getNullMapColumnPtr();
@ -247,7 +247,7 @@ ColumnPtr wrapInNullable(const ColumnPtr & src, const ColumnsWithTypeAndName & a
if (isColumnConst(*elem.column))
continue;
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*elem.column))
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(&*elem.column))
{
const ColumnPtr & null_map_column = nullable->getNullMapColumnPtr();
if (!result_null_map_column)

View File

@ -15,6 +15,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
class IFunction;
/// Methods, that helps dispatching over real column types.
@ -25,6 +30,13 @@ const Type * checkAndGetDataType(const IDataType * data_type)
return typeid_cast<const Type *>(data_type);
}
/// Throws on mismatch.
template <typename Type>
const Type & checkAndGetDataType(const IDataType & data_type)
{
return typeid_cast<const Type &>(data_type);
}
template <typename... Types>
bool checkDataTypes(const IDataType * data_type)
{
@ -34,10 +46,12 @@ bool checkDataTypes(const IDataType * data_type)
template <typename Type>
const ColumnConst * checkAndGetColumnConst(const IColumn * column)
{
if (!column || !isColumnConst(*column))
if (!column)
return {};
const ColumnConst * res = assert_cast<const ColumnConst *>(column);
const ColumnConst * res = checkAndGetColumn<ColumnConst>(column);
if (!res)
return {};
if (!checkColumn<Type>(&res->getDataColumn()))
return {};
@ -45,6 +59,18 @@ const ColumnConst * checkAndGetColumnConst(const IColumn * column)
return res;
}
template <typename Type>
const ColumnConst & checkAndGetColumnConst(const IColumn & column)
{
const ColumnConst & res = checkAndGetColumn<ColumnConst>(column);
const auto & data_column = res.getDataColumn();
if (!checkColumn<Type>(&data_column))
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unexpected const column type: expected {}, got {}", demangle(typeid(Type).name()), demangle(typeid(data_column).name()));
return res;
}
template <typename Type>
const Type * checkAndGetColumnConstData(const IColumn * column)
{

View File

@ -140,7 +140,7 @@ public:
const auto & src = arguments[0];
const auto & col = *src.column;
if (!checkAndGetColumn<ColumnVector<T>>(col))
if (!checkAndGetColumn<ColumnVector<T>>(&col))
return false;
auto & result_data = result_column->getData();

View File

@ -193,8 +193,8 @@ private:
const ColumnArray * array = typeid_cast<const ColumnArray *>(arguments[0].column.get());
const ColumnPtr & mapped = array->getDataPtr();
const ColumnArray::Offsets & offsets = array->getOffsets();
const ColumnVector<T> * column = checkAndGetColumn<ColumnVector<T>>(&*mapped);
const typename ColumnVector<T>::Container & input_data = column->getData();
const ColumnVector<T> & column = checkAndGetColumn<ColumnVector<T>>(*mapped);
const typename ColumnVector<T>::Container & input_data = column.getData();
// output data
Array params_row;

View File

@ -536,7 +536,7 @@ public:
const auto & col_type_name = arguments[0];
const ColumnPtr & column = col_type_name.column;
if (const auto * col_in = checkAndGetColumn<ColumnIPv4>(*column))
if (const auto * col_in = checkAndGetColumn<ColumnIPv4>(&*column))
{
auto col_res = ColumnIPv6::create();
@ -551,7 +551,7 @@ public:
return col_res;
}
if (const auto * col_in = checkAndGetColumn<ColumnUInt32>(*column))
if (const auto * col_in = checkAndGetColumn<ColumnUInt32>(&*column))
{
auto col_res = ColumnFixedString::create(IPV6_BINARY_LENGTH);

View File

@ -3302,7 +3302,7 @@ private:
/// both columns have type UInt8, but we shouldn't use identity wrapper,
/// because Bool column can contain only 0 and 1.
auto res_column = to_type->createColumn();
const auto & data_from = checkAndGetColumn<ColumnUInt8>(arguments[0].column.get())->getData();
const auto & data_from = checkAndGetColumn<ColumnUInt8>(*arguments[0].column).getData();
auto & data_to = assert_cast<ColumnUInt8 *>(res_column.get())->getData();
data_to.resize(data_from.size());
for (size_t i = 0; i != data_from.size(); ++i)

View File

@ -467,28 +467,28 @@ struct Dispatcher
static ColumnPtr apply(const IColumn * col_general, Scale scale_arg)
{
const auto * const col = checkAndGetColumn<ColumnVector<T>>(col_general);
const auto & col = checkAndGetColumn<ColumnVector<T>>(*col_general);
auto col_res = ColumnVector<T>::create();
typename ColumnVector<T>::Container & vec_res = col_res->getData();
vec_res.resize(col->getData().size());
vec_res.resize(col.getData().size());
if (!vec_res.empty())
{
if (scale_arg == 0)
{
size_t scale = 1;
FunctionRoundingImpl<ScaleMode::Zero>::apply(col->getData(), scale, vec_res);
FunctionRoundingImpl<ScaleMode::Zero>::apply(col.getData(), scale, vec_res);
}
else if (scale_arg > 0)
{
size_t scale = intExp10(scale_arg);
FunctionRoundingImpl<ScaleMode::Positive>::apply(col->getData(), scale, vec_res);
FunctionRoundingImpl<ScaleMode::Positive>::apply(col.getData(), scale, vec_res);
}
else
{
size_t scale = intExp10(-scale_arg);
FunctionRoundingImpl<ScaleMode::Negative>::apply(col->getData(), scale, vec_res);
FunctionRoundingImpl<ScaleMode::Negative>::apply(col.getData(), scale, vec_res);
}
}
@ -502,14 +502,14 @@ struct Dispatcher<T, rounding_mode, tie_breaking_mode>
public:
static ColumnPtr apply(const IColumn * col_general, Scale scale_arg)
{
const auto * const col = checkAndGetColumn<ColumnDecimal<T>>(col_general);
const typename ColumnDecimal<T>::Container & vec_src = col->getData();
const auto & col = checkAndGetColumn<ColumnDecimal<T>>(*col_general);
const typename ColumnDecimal<T>::Container & vec_src = col.getData();
auto col_res = ColumnDecimal<T>::create(vec_src.size(), col->getScale());
auto col_res = ColumnDecimal<T>::create(vec_src.size(), col.getScale());
auto & vec_res = col_res->getData();
if (!vec_res.empty())
DecimalRoundingImpl<T, rounding_mode, tie_breaking_mode>::apply(col->getData(), col->getScale(), vec_res, scale_arg);
DecimalRoundingImpl<T, rounding_mode, tie_breaking_mode>::apply(col.getData(), col.getScale(), vec_res, scale_arg);
return col_res;
}

View File

@ -153,8 +153,8 @@ public:
auto col_res = ColumnVector<UInt64>::create();
auto & vec_res = col_res->getData();
vec_res.resize(column->size());
const ColumnString * col_str_vector = checkAndGetColumn<ColumnString>(&*column);
Impl::apply(col_str_vector->getChars(), col_str_vector->getOffsets(), shingle_size, vec_res);
const ColumnString & col_str_vector = checkAndGetColumn<ColumnString>(*column);
Impl::apply(col_str_vector.getChars(), col_str_vector.getOffsets(), shingle_size, vec_res);
return col_res;
}
else if constexpr (is_arg) // Min hash arg
@ -170,8 +170,8 @@ public:
auto min_tuple = ColumnTuple::create(std::move(min_columns));
auto max_tuple = ColumnTuple::create(std::move(max_columns));
const ColumnString * col_str_vector = checkAndGetColumn<ColumnString>(&*column);
Impl::apply(col_str_vector->getChars(), col_str_vector->getOffsets(), shingle_size, num_hashes, nullptr, nullptr, min_tuple.get(), max_tuple.get());
const ColumnString & col_str_vector = checkAndGetColumn<ColumnString>(*column);
Impl::apply(col_str_vector.getChars(), col_str_vector.getOffsets(), shingle_size, num_hashes, nullptr, nullptr, min_tuple.get(), max_tuple.get());
MutableColumns tuple_columns;
tuple_columns.emplace_back(std::move(min_tuple));
@ -186,8 +186,8 @@ public:
auto & vec_h2 = col_h2->getData();
vec_h1.resize(column->size());
vec_h2.resize(column->size());
const ColumnString * col_str_vector = checkAndGetColumn<ColumnString>(&*column);
Impl::apply(col_str_vector->getChars(), col_str_vector->getOffsets(), shingle_size, num_hashes, &vec_h1, &vec_h2, nullptr, nullptr);
const ColumnString & col_str_vector = checkAndGetColumn<ColumnString>(*column);
Impl::apply(col_str_vector.getChars(), col_str_vector.getOffsets(), shingle_size, num_hashes, &vec_h1, &vec_h2, nullptr, nullptr);
MutableColumns tuple_columns;
tuple_columns.emplace_back(std::move(col_h1));
tuple_columns.emplace_back(std::move(col_h2));

View File

@ -158,12 +158,12 @@ public:
auto out_tmp = ColumnArray::create(nested_types[i]->createColumn());
size_t array_size = tuple_coulmn->size();
const auto * arr = checkAndGetColumn<ColumnArray>(tuple_coulmn.get());
const auto & arr = checkAndGetColumn<ColumnArray>(*tuple_coulmn);
for (size_t j = 0; j < array_size; ++j)
{
Field arr_field;
arr->get(j, arr_field);
arr.get(j, arr_field);
out_tmp->insert(arr_field);
}

View File

@ -185,7 +185,7 @@ struct MultiMatchAllIndicesImpl
size_t prev_haystack_offset = 0;
size_t prev_needles_offset = 0;
const ColumnString * needles_data_string = checkAndGetColumn<ColumnString>(&needles_data);
const ColumnString & needles_data_string = checkAndGetColumn<ColumnString>(needles_data);
std::vector<std::string_view> needles;
@ -195,7 +195,7 @@ struct MultiMatchAllIndicesImpl
for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j)
{
needles.emplace_back(needles_data_string->getDataAt(j).toView());
needles.emplace_back(needles_data_string.getDataAt(j).toView());
}
if (needles.empty())

View File

@ -212,7 +212,7 @@ struct MultiMatchAnyImpl
size_t prev_haystack_offset = 0;
size_t prev_needles_offset = 0;
const ColumnString * needles_data_string = checkAndGetColumn<ColumnString>(&needles_data);
const ColumnString & needles_data_string = checkAndGetColumn<ColumnString>(needles_data);
std::vector<std::string_view> needles;
@ -221,7 +221,7 @@ struct MultiMatchAnyImpl
needles.reserve(needles_offsets[i] - prev_needles_offset);
for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j)
needles.emplace_back(needles_data_string->getDataAt(j).toView());
needles.emplace_back(needles_data_string.getDataAt(j).toView());
if (needles.empty())
{

View File

@ -89,7 +89,7 @@ struct MultiSearchAllPositionsImpl
offsets_res.reserve(haystack_offsets.size());
const ColumnString * needles_data_string = checkAndGetColumn<ColumnString>(&needles_data);
const ColumnString & needles_data_string = checkAndGetColumn<ColumnString>(needles_data);
std::vector<std::string_view> needles;
@ -99,7 +99,7 @@ struct MultiSearchAllPositionsImpl
for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j)
{
needles.emplace_back(needles_data_string->getDataAt(j).toView());
needles.emplace_back(needles_data_string.getDataAt(j).toView());
}
const size_t needles_size = needles.size();

View File

@ -88,7 +88,7 @@ struct MultiSearchFirstIndexImpl
size_t prev_haystack_offset = 0;
size_t prev_needles_offset = 0;
const ColumnString * needles_data_string = checkAndGetColumn<ColumnString>(&needles_data);
const ColumnString & needles_data_string = checkAndGetColumn<ColumnString>(needles_data);
std::vector<std::string_view> needles;
@ -98,7 +98,7 @@ struct MultiSearchFirstIndexImpl
for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j)
{
needles.emplace_back(needles_data_string->getDataAt(j).toView());
needles.emplace_back(needles_data_string.getDataAt(j).toView());
}
auto searcher = Impl::createMultiSearcherInBigHaystack(needles); // sub-optimal

View File

@ -97,7 +97,7 @@ struct MultiSearchFirstPositionImpl
size_t prev_haystack_offset = 0;
size_t prev_needles_offset = 0;
const ColumnString * needles_data_string = checkAndGetColumn<ColumnString>(&needles_data);
const ColumnString & needles_data_string = checkAndGetColumn<ColumnString>(needles_data);
std::vector<std::string_view> needles;
@ -112,7 +112,7 @@ struct MultiSearchFirstPositionImpl
for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j)
{
needles.emplace_back(needles_data_string->getDataAt(j).toView());
needles.emplace_back(needles_data_string.getDataAt(j).toView());
}
auto searcher = Impl::createMultiSearcherInBigHaystack(needles); // sub-optimal

View File

@ -87,7 +87,7 @@ struct MultiSearchImpl
size_t prev_haystack_offset = 0;
size_t prev_needles_offset = 0;
const ColumnString * needles_data_string = checkAndGetColumn<ColumnString>(&needles_data);
const ColumnString & needles_data_string = checkAndGetColumn<ColumnString>(needles_data);
std::vector<std::string_view> needles;
@ -97,7 +97,7 @@ struct MultiSearchImpl
for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j)
{
needles.emplace_back(needles_data_string->getDataAt(j).toView());
needles.emplace_back(needles_data_string.getDataAt(j).toView());
}
const auto * const haystack = &haystack_data[prev_haystack_offset];

View File

@ -69,7 +69,7 @@ public:
const ColumnConst * column_tld_list_name = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get());
FirstSignificantSubdomainCustomLookup tld_lookup(column_tld_list_name->getValue<String>());
if (const ColumnString * col = checkAndGetColumn<ColumnString>(*arguments[0].column))
if (const ColumnString * col = checkAndGetColumn<ColumnString>(&*arguments[0].column))
{
auto col_res = ColumnString::create();
vector(tld_lookup, col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets());

View File

@ -80,14 +80,14 @@ namespace
const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC");
if (WhichDataType(arg1.type).isDateTime())
{
const auto * date_time_col = checkAndGetColumn<ColumnDateTime>(arg1.column.get());
size_t col_size = date_time_col->size();
const auto & date_time_col = checkAndGetColumn<ColumnDateTime>(*arg1.column);
size_t col_size = date_time_col.size();
using ColVecTo = DataTypeDateTime::ColumnType;
typename ColVecTo::MutablePtr result_column = ColVecTo::create(col_size);
typename ColVecTo::Container & result_data = result_column->getData();
for (size_t i = 0; i < col_size; ++i)
{
UInt32 date_time_val = date_time_col->getElement(i);
UInt32 date_time_val = date_time_col.getElement(i);
LocalDateTime date_time(date_time_val, Name::to ? utc_time_zone : DateLUT::instance(time_zone_val));
time_t time_val = date_time.to_time_t(Name::from ? utc_time_zone : DateLUT::instance(time_zone_val));
result_data[i] = static_cast<UInt32>(time_val);
@ -96,8 +96,8 @@ namespace
}
else if (WhichDataType(arg1.type).isDateTime64())
{
const auto * date_time_col = checkAndGetColumn<ColumnDateTime64>(arg1.column.get());
size_t col_size = date_time_col->size();
const auto & date_time_col = checkAndGetColumn<ColumnDateTime64>(*arg1.column);
size_t col_size = date_time_col.size();
const DataTypeDateTime64 * date_time_type = static_cast<const DataTypeDateTime64 *>(arg1.type.get());
UInt32 col_scale = date_time_type->getScale();
Int64 scale_multiplier = DecimalUtils::scaleMultiplier<Int64>(col_scale);
@ -106,7 +106,7 @@ namespace
typename ColDecimalTo::Container & result_data = result_column->getData();
for (size_t i = 0; i < col_size; ++i)
{
DateTime64 date_time_val = date_time_col->getElement(i);
DateTime64 date_time_val = date_time_col.getElement(i);
Int64 seconds = date_time_val.value / scale_multiplier;
Int64 micros = date_time_val.value % scale_multiplier;
LocalDateTime date_time(seconds, Name::to ? utc_time_zone : DateLUT::instance(time_zone_val));

View File

@ -317,7 +317,7 @@ public:
ErrorCodes::ILLEGAL_COLUMN, "Expected Array column, found {}", column_array_ptr->getName());
column_array_ptr = recursiveRemoveLowCardinality(column_const_array->convertToFullColumn());
column_array = checkAndGetColumn<ColumnArray>(column_array_ptr.get());
column_array = &checkAndGetColumn<ColumnArray>(*column_array_ptr);
}
if (!array_type)

View File

@ -34,7 +34,7 @@ struct ArrayCompactImpl
using ColVecType = ColumnVectorOrDecimal<T>;
const ColVecType * check_values_column = checkAndGetColumn<ColVecType>(mapped.get());
const ColVecType * src_values_column = checkAndGetColumn<ColVecType>(array.getData());
const ColVecType * src_values_column = checkAndGetColumn<ColVecType>(&array.getData());
if (!src_values_column || !check_values_column)
return false;

View File

@ -89,20 +89,20 @@ private:
ColumnPtr FunctionArrayDistinct::executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const
{
ColumnPtr array_ptr = arguments[0].column;
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
const ColumnArray & array = checkAndGetColumn<ColumnArray>(*array_ptr);
const auto & return_type = result_type;
auto res_ptr = return_type->createColumn();
ColumnArray & res = assert_cast<ColumnArray &>(*res_ptr);
const IColumn & src_data = array->getData();
const ColumnArray::Offsets & offsets = array->getOffsets();
const IColumn & src_data = array.getData();
const ColumnArray::Offsets & offsets = array.getOffsets();
IColumn & res_data = res.getData();
ColumnArray::Offsets & res_offsets = res.getOffsets();
const ColumnNullable * nullable_col = checkAndGetColumn<ColumnNullable>(src_data);
const ColumnNullable * nullable_col = checkAndGetColumn<ColumnNullable>(&src_data);
const IColumn * inner_col;

View File

@ -1538,9 +1538,9 @@ ColumnPtr FunctionArrayElement::executeMap2(const ColumnsWithTypeAndName & argum
return nullptr;
const ColumnArray * col_map_nested = &col_map->getNestedColumn();
const ColumnTuple * col_map_kv = checkAndGetColumn<ColumnTuple>(col_map_nested->getDataPtr().get());
ColumnPtr col_map_keys = col_map_kv->getColumnPtr(0);
ColumnPtr col_map_values = col_map_kv->getColumnPtr(1);
const ColumnTuple & col_map_kv = checkAndGetColumn<ColumnTuple>(*col_map_nested->getDataPtr());
ColumnPtr col_map_keys = col_map_kv.getColumnPtr(0);
ColumnPtr col_map_values = col_map_kv.getColumnPtr(1);
const DataTypeMap & map_type
= typeid_cast<const DataTypeMap &>(*typeid_cast<const DataTypeArray &>(*arguments[0].type).getNestedType());

View File

@ -165,7 +165,7 @@ ColumnPtr FunctionArrayEnumerateExtended<Derived>::executeImpl(const ColumnsWith
for (size_t i = 0; i < num_arguments; ++i)
{
if (const auto * nullable_col = checkAndGetColumn<ColumnNullable>(*data_columns[i]))
if (const auto * nullable_col = checkAndGetColumn<ColumnNullable>(data_columns[i]))
{
if (num_arguments == 1)
data_columns[i] = &nullable_col->getNestedColumn();

View File

@ -506,10 +506,10 @@ private:
const ColumnNullable * nullable = nullptr;
if (col_array)
nullable = checkAndGetColumn<ColumnNullable>(col_array->getData());
nullable = checkAndGetColumn<ColumnNullable>(&col_array->getData());
const auto & arg_column = arguments[1].column;
const ColumnNullable * arg_nullable = checkAndGetColumn<ColumnNullable>(*arg_column);
const ColumnNullable * arg_nullable = checkAndGetColumn<ColumnNullable>(&*arg_column);
if (!nullable && !arg_nullable)
{
@ -738,7 +738,7 @@ private:
const auto [null_map_data, null_map_item] = getNullMaps(arguments);
if (const ColumnConst * col_arg_const = checkAndGetColumn<ColumnConst>(*arguments[1].column))
if (const ColumnConst * col_arg_const = checkAndGetColumn<ColumnConst>(&*arguments[1].column))
{
const IColumnUnique & col_lc_dict = col_lc->getDictionary();
@ -754,7 +754,7 @@ private:
if (!col_arg_cloned->isNullAt(0))
{
if (col_arg_cloned->isNullable())
col_arg_cloned = checkAndGetColumn<ColumnNullable>(*col_arg_cloned)->getNestedColumnPtr();
col_arg_cloned = checkAndGetColumn<ColumnNullable>(*col_arg_cloned).getNestedColumnPtr();
StringRef elem = col_arg_cloned->getDataAt(0);
@ -786,7 +786,7 @@ private:
else if (col_lc->nestedIsNullable()) // LowCardinality(Nullable(T)) and U
{
const ColumnPtr left_casted = col_lc->convertToFullColumnIfLowCardinality(); // Nullable(T)
const ColumnNullable& left_nullable = *checkAndGetColumn<ColumnNullable>(left_casted.get());
const ColumnNullable & left_nullable = checkAndGetColumn<ColumnNullable>(*left_casted);
const NullMap * const null_map_left_casted = &left_nullable.getNullMapColumn().getData();

View File

@ -97,8 +97,8 @@ public:
{
if (const ColumnConst * col_const = typeid_cast<const ColumnConst *>(col.column.get()))
{
const ColumnArray * col_const_array = checkAndGetColumn<ColumnArray>(col_const->getDataColumnPtr().get());
return {col_const_array, true};
const ColumnArray & col_const_array = checkAndGetColumn<ColumnArray>(*col_const->getDataColumnPtr());
return {&col_const_array, true};
}
else if (const ColumnArray * col_non_const_array = checkAndGetColumn<ColumnArray>(col.column.get()))
return {col_non_const_array, false};
@ -128,8 +128,8 @@ public:
vectorWithEmptyIntersect<left_is_const, right_is_const>(left_array->getOffsets(), right_array->getOffsets(), vec_res); \
else \
{ \
const ColumnArray * intersect_column_array = checkAndGetColumn<ColumnArray>(intersect_column.column.get()); \
vector<left_is_const, right_is_const>(intersect_column_array->getOffsets(), left_array->getOffsets(), right_array->getOffsets(), vec_res); \
const ColumnArray & intersect_column_array = checkAndGetColumn<ColumnArray>(*intersect_column.column); \
vector<left_is_const, right_is_const>(intersect_column_array.getOffsets(), left_array->getOffsets(), right_array->getOffsets(), vec_res); \
}
if (!left_is_const && !right_is_const)

View File

@ -162,7 +162,7 @@ ColumnPtr FunctionArrayUniq::executeImpl(const ColumnsWithTypeAndName & argument
for (size_t i = 0; i < num_arguments; ++i)
{
if (const auto * nullable_col = checkAndGetColumn<ColumnNullable>(*data_columns[i]))
if (const auto * nullable_col = checkAndGetColumn<ColumnNullable>(data_columns[i]))
{
if (num_arguments == 1)
data_columns[i] = &nullable_col->getNestedColumn();

View File

@ -391,7 +391,7 @@ ColumnPtr FunctionEmptyArrayToSingle::executeImpl(const ColumnsWithTypeAndName &
const IColumn * inner_col;
IColumn * inner_res_col;
const auto * nullable_col = checkAndGetColumn<ColumnNullable>(src_data);
const auto * nullable_col = checkAndGetColumn<ColumnNullable>(&src_data);
if (nullable_col)
{
inner_col = &nullable_col->getNestedColumn();

View File

@ -404,7 +404,7 @@ private:
{
if (!col.type->isNullable())
return;
const ColumnNullable * nullable_col = checkAndGetColumn<ColumnNullable>(*col.column);
const ColumnNullable * nullable_col = checkAndGetColumn<ColumnNullable>(col.column.get());
if (!nullable_col)
nullable_col = checkAndGetColumnConstData<ColumnNullable>(col.column.get());
if (!nullable_col)
@ -421,8 +421,8 @@ private:
const auto * col = arguments[0].column.get();
if (arguments[0].type->isNullable())
{
const auto * nullable = checkAndGetColumn<ColumnNullable>(*arguments[0].column);
col = nullable->getNestedColumnPtr().get();
const auto & nullable = checkAndGetColumn<ColumnNullable>(*arguments[0].column);
col = nullable.getNestedColumnPtr().get();
}
if (!((res = executeInternal<UInt8>(col)) || (res = executeInternal<UInt16>(col)) || (res = executeInternal<UInt32>(col))

View File

@ -183,7 +183,7 @@ public:
const ColumnString & col_string = assert_cast<const ColumnString &>(*str_subcolumn.get());
auto col_res = ColumnString::create();
if (const ColumnNullable * col_nullable = checkAndGetColumn<ColumnNullable>(col_arr.getData()))
if (const ColumnNullable * col_nullable = checkAndGetColumn<ColumnNullable>(&col_arr.getData()))
executeInternal(col_string, col_arr, delimiter, *col_res, col_nullable->getNullMapData().data());
else
executeInternal(col_string, col_arr, delimiter, *col_res);

View File

@ -54,7 +54,7 @@ public:
if (arguments[0].type->onlyNull() && !col->empty())
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot create non-empty column with type Nothing");
if (const auto * nullable_col = checkAndGetColumn<ColumnNullable>(*col))
if (const auto * nullable_col = checkAndGetColumn<ColumnNullable>(&*col))
return nullable_col->getNestedColumnPtr();
else
return col;

View File

@ -157,12 +157,12 @@ public:
/// if last argument is not nullable, result should be also not nullable
if (!multi_if_args.back().column->isNullable() && res->isNullable())
{
if (const auto * column_lc = checkAndGetColumn<ColumnLowCardinality>(*res))
res = checkAndGetColumn<ColumnNullable>(*column_lc->convertToFullColumn())->getNestedColumnPtr();
else if (const auto * column_const = checkAndGetColumn<ColumnConst>(*res))
res = checkAndGetColumn<ColumnNullable>(column_const->getDataColumn())->getNestedColumnPtr();
if (const auto * column_lc = checkAndGetColumn<ColumnLowCardinality>(&*res))
res = checkAndGetColumn<ColumnNullable>(*column_lc->convertToFullColumn()).getNestedColumnPtr();
else if (const auto * column_const = checkAndGetColumn<ColumnConst>(&*res))
res = checkAndGetColumn<ColumnNullable>(column_const->getDataColumn()).getNestedColumnPtr();
else
res = checkAndGetColumn<ColumnNullable>(*res)->getNestedColumnPtr();
res = checkAndGetColumn<ColumnNullable>(&*res)->getNestedColumnPtr();
}
return res;

View File

@ -34,8 +34,8 @@ namespace DB
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
using ColVecType = typename FromDataType::ColumnType;
const ColVecType * col_from = checkAndGetColumn<ColVecType>(arguments[0].column.get());
const typename ColVecType::Container & vec_from = col_from->getData();
const ColVecType & col_from = checkAndGetColumn<ColVecType>(*arguments[0].column);
const typename ColVecType::Container & vec_from = col_from.getData();
auto col_to = ColumnString::create();
ColumnString::Chars & data_to = col_to->getChars();

View File

@ -59,7 +59,7 @@ public:
template <typename AggregationKeyChecker>
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, size_t input_rows_count, AggregationKeyChecker checker) const
{
const auto * grouping_set_column = checkAndGetColumn<ColumnUInt64>(arguments[0].column.get());
const auto & grouping_set_column = checkAndGetColumn<ColumnUInt64>(*arguments[0].column);
auto result = ColumnUInt64::create();
auto & result_data = result->getData();
@ -68,7 +68,7 @@ public:
const auto * result_table = likely(force_compatibility) ? COMPATIBLE_MODE : INCOMPATIBLE_MODE;
for (size_t i = 0; i < input_rows_count; ++i)
{
UInt64 set_index = grouping_set_column->getElement(i);
UInt64 set_index = grouping_set_column.getElement(i);
UInt64 value = 0;
for (auto index : arguments_indexes)

View File

@ -88,8 +88,8 @@ ColumnPtr FunctionHasColumnInTable::executeImpl(const ColumnsWithTypeAndName & a
{
auto get_string_from_columns = [&](const ColumnWithTypeAndName & column) -> String
{
const ColumnConst * const_column = checkAndGetColumnConst<ColumnString>(column.column.get());
return const_column->getValue<String>();
const ColumnConst & const_column = checkAndGetColumnConst<ColumnString>(*column.column);
return const_column.getValue<String>();
};
size_t arg = 0;

View File

@ -948,12 +948,12 @@ private:
bool cond_is_const = false;
bool cond_is_true = false;
bool cond_is_false = false;
if (const auto * const_arg = checkAndGetColumn<ColumnConst>(*arg_cond.column))
if (const auto * const_arg = checkAndGetColumn<ColumnConst>(&*arg_cond.column))
{
cond_is_const = true;
not_const_condition = const_arg->getDataColumnPtr();
ColumnPtr data_column = const_arg->getDataColumnPtr();
if (const auto * const_nullable_arg = checkAndGetColumn<ColumnNullable>(*data_column))
if (const auto * const_nullable_arg = checkAndGetColumn<ColumnNullable>(&*data_column))
{
data_column = const_nullable_arg->getNestedColumnPtr();
if (!data_column->empty())
@ -962,7 +962,7 @@ private:
if (!data_column->empty())
{
cond_is_true = !cond_is_null && checkAndGetColumn<ColumnUInt8>(*data_column)->getBool(0);
cond_is_true = !cond_is_null && checkAndGetColumn<ColumnUInt8>(*data_column).getBool(0);
cond_is_false = !cond_is_null && !cond_is_true;
}
}
@ -975,12 +975,12 @@ private:
else if (cond_is_false || cond_is_null)
return castColumn(column2, result_type);
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*not_const_condition))
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(&*not_const_condition))
{
ColumnPtr new_cond_column = nullable->getNestedColumnPtr();
size_t column_size = arg_cond.column->size();
if (checkAndGetColumn<ColumnUInt8>(*new_cond_column))
if (checkAndGetColumn<ColumnUInt8>(&*new_cond_column))
{
auto nested_column_copy = new_cond_column->cloneResized(new_cond_column->size());
typeid_cast<ColumnUInt8 *>(nested_column_copy.get())->applyZeroMap(nullable->getNullMapData());
@ -1027,12 +1027,12 @@ private:
/// Const(size = 0, Int32(size = 1))
static ColumnPtr recursiveGetNestedColumnWithoutNullable(const ColumnPtr & column)
{
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*column))
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(&*column))
{
/// Nullable cannot contain Nullable
return nullable->getNestedColumnPtr();
}
else if (const auto * column_const = checkAndGetColumn<ColumnConst>(*column))
else if (const auto * column_const = checkAndGetColumn<ColumnConst>(&*column))
{
/// Save Constant, but remove Nullable
return ColumnConst::create(recursiveGetNestedColumnWithoutNullable(column_const->getDataColumnPtr()), column->size());
@ -1051,8 +1051,8 @@ private:
const ColumnWithTypeAndName & arg_then = arguments[1];
const ColumnWithTypeAndName & arg_else = arguments[2];
const auto * then_is_nullable = checkAndGetColumn<ColumnNullable>(*arg_then.column);
const auto * else_is_nullable = checkAndGetColumn<ColumnNullable>(*arg_else.column);
const auto * then_is_nullable = checkAndGetColumn<ColumnNullable>(&*arg_then.column);
const auto * else_is_nullable = checkAndGetColumn<ColumnNullable>(&*arg_else.column);
if (!then_is_nullable && !else_is_nullable)
return nullptr;

View File

@ -46,7 +46,7 @@ public:
if (isVariant(elem.type))
{
const auto & discriminators = checkAndGetColumn<ColumnVariant>(*elem.column)->getLocalDiscriminators();
const auto & discriminators = checkAndGetColumn<ColumnVariant>(*elem.column).getLocalDiscriminators();
auto res = DataTypeUInt8().createColumn();
auto & data = typeid_cast<ColumnUInt8 &>(*res).getData();
data.resize(discriminators.size());
@ -57,17 +57,17 @@ public:
if (elem.type->isLowCardinalityNullable())
{
const auto * low_cardinality_column = checkAndGetColumn<ColumnLowCardinality>(*elem.column);
const size_t null_index = low_cardinality_column->getDictionary().getNullValueIndex();
const auto & low_cardinality_column = checkAndGetColumn<ColumnLowCardinality>(*elem.column);
const size_t null_index = low_cardinality_column.getDictionary().getNullValueIndex();
auto res = DataTypeUInt8().createColumn();
auto & data = typeid_cast<ColumnUInt8 &>(*res).getData();
data.resize(low_cardinality_column->size());
for (size_t i = 0; i != low_cardinality_column->size(); ++i)
data[i] = (low_cardinality_column->getIndexAt(i) != null_index);
data.resize(low_cardinality_column.size());
for (size_t i = 0; i != low_cardinality_column.size(); ++i)
data[i] = (low_cardinality_column.getIndexAt(i) != null_index);
return res;
}
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*elem.column))
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(&*elem.column))
{
/// Return the negated null map.
auto res_column = ColumnUInt8::create(input_rows_count);

View File

@ -48,7 +48,7 @@ public:
if (isVariant(elem.type))
{
const auto & discriminators = checkAndGetColumn<ColumnVariant>(*elem.column)->getLocalDiscriminators();
const auto & discriminators = checkAndGetColumn<ColumnVariant>(*elem.column).getLocalDiscriminators();
auto res = DataTypeUInt8().createColumn();
auto & data = typeid_cast<ColumnUInt8 &>(*res).getData();
data.reserve(discriminators.size());
@ -59,17 +59,17 @@ public:
if (elem.type->isLowCardinalityNullable())
{
const auto * low_cardinality_column = checkAndGetColumn<ColumnLowCardinality>(*elem.column);
size_t null_index = low_cardinality_column->getDictionary().getNullValueIndex();
const auto & low_cardinality_column = checkAndGetColumn<ColumnLowCardinality>(*elem.column);
size_t null_index = low_cardinality_column.getDictionary().getNullValueIndex();
auto res = DataTypeUInt8().createColumn();
auto & data = typeid_cast<ColumnUInt8 &>(*res).getData();
data.reserve(low_cardinality_column->size());
for (size_t i = 0; i != low_cardinality_column->size(); ++i)
data.push_back(low_cardinality_column->getIndexAt(i) == null_index);
data.reserve(low_cardinality_column.size());
for (size_t i = 0; i != low_cardinality_column.size(); ++i)
data.push_back(low_cardinality_column.getIndexAt(i) == null_index);
return res;
}
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*elem.column))
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(&*elem.column))
{
/// Merely return the embedded null map.
return nullable->getNullMapColumnPtr();

View File

@ -102,14 +102,14 @@ struct ContinuousImpl
auto baseline_argument = arguments[0];
baseline_argument.column = baseline_argument.column->convertToFullColumnIfConst();
auto baseline_column_untyped = castColumnAccurate(baseline_argument, float_64_type);
const auto * baseline_column = checkAndGetColumn<ColumnVector<Float64>>(*baseline_column_untyped);
const auto & baseline_column_data = baseline_column->getData();
const auto & baseline_column = checkAndGetColumn<ColumnVector<Float64>>(*baseline_column_untyped);
const auto & baseline_column_data = baseline_column.getData();
auto sigma_argument = arguments[1];
sigma_argument.column = sigma_argument.column->convertToFullColumnIfConst();
auto sigma_column_untyped = castColumnAccurate(sigma_argument, float_64_type);
const auto * sigma_column = checkAndGetColumn<ColumnVector<Float64>>(*sigma_column_untyped);
const auto & sigma_column_data = sigma_column->getData();
const auto & sigma_column = checkAndGetColumn<ColumnVector<Float64>>(*sigma_column_untyped);
const auto & sigma_column_data = sigma_column.getData();
const IColumn & col_mde = *arguments[2].column;
const IColumn & col_power = *arguments[3].column;

View File

@ -198,7 +198,7 @@ public:
if (cond_col->onlyNull())
continue;
if (const auto * column_const = checkAndGetColumn<ColumnConst>(*cond_col))
if (const auto * column_const = checkAndGetColumn<ColumnConst>(&*cond_col))
{
Field value = column_const->getField();

View File

@ -51,14 +51,14 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
{
const auto * column_string = checkAndGetColumn<ColumnString>(arguments[0].column.get());
const auto & column_string = checkAndGetColumn<ColumnString>(*arguments[0].column);
Serializer serializer;
Geometry geometry;
for (size_t i = 0; i < input_rows_count; ++i)
{
const auto & str = column_string->getDataAt(i).toString();
const auto & str = column_string.getDataAt(i).toString();
boost::geometry::read_wkt(str, geometry);
serializer.add(geometry);
}

View File

@ -238,9 +238,9 @@ public:
{
using DataType = std::decay_t<decltype(type)>;
using T = typename DataType::FieldType;
const ColumnVector<T> * column = checkAndGetColumn<ColumnVector<T>>(col_num.get());
const ColumnVector<T> & column = checkAndGetColumn<ColumnVector<T>>(*col_num);
auto col_res = ColumnString::create();
RepeatImpl::vectorStrVectorRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), column->getData());
RepeatImpl::vectorStrVectorRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), column.getData());
res = std::move(col_res);
return true;
}))
@ -258,9 +258,9 @@ public:
{
using DataType = std::decay_t<decltype(type)>;
using T = typename DataType::FieldType;
const ColumnVector<T> * column = checkAndGetColumn<ColumnVector<T>>(col_num.get());
const ColumnVector<T> & column = checkAndGetColumn<ColumnVector<T>>(*col_num);
auto col_res = ColumnString::create();
RepeatImpl::constStrVectorRepeat(copy_str, col_res->getChars(), col_res->getOffsets(), column->getData());
RepeatImpl::constStrVectorRepeat(copy_str, col_res->getChars(), col_res->getOffsets(), column.getData());
res = std::move(col_res);
return true;
}))

View File

@ -61,10 +61,10 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
ColumnPtr col = arguments[0].column;
const ColumnArray * col_arr = checkAndGetColumn<ColumnArray>(col.get());
const ColumnArray & col_arr = checkAndGetColumn<ColumnArray>(*col);
const IColumn & arr_data = col_arr->getData();
const ColumnArray::Offsets & arr_offsets = col_arr->getOffsets();
const IColumn & arr_data = col_arr.getData();
const ColumnArray::Offsets & arr_offsets = col_arr.getOffsets();
ColumnPtr col_res;
if (input_rows_count == 0)

View File

@ -61,10 +61,10 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
ColumnPtr array_ptr = arguments[0].column;
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
const ColumnArray & array = checkAndGetColumn<ColumnArray>(*array_ptr);
const IColumn & src_data = array->getData();
const ColumnArray::Offsets & offsets = array->getOffsets();
const IColumn & src_data = array.getData();
const ColumnArray::Offsets & offsets = array.getOffsets();
auto res = ColumnFloat64::create(input_rows_count);
auto & res_data = res->getData();

View File

@ -57,14 +57,14 @@ public:
template <typename DataType>
bool executeConstant(ColumnPtr col_times, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars) const
{
const ColumnConst * col_times_const = checkAndGetColumn<ColumnConst>(col_times.get());
const ColumnConst & col_times_const = checkAndGetColumn<ColumnConst>(*col_times);
const ColumnPtr & col_times_const_internal = col_times_const->getDataColumnPtr();
const ColumnPtr & col_times_const_internal = col_times_const.getDataColumnPtr();
if (!checkAndGetColumn<typename DataType::ColumnType>(col_times_const_internal.get()))
return false;
using T = typename DataType::FieldType;
T times = col_times_const->getValue<T>();
T times = col_times_const.getValue<T>();
if (times < 1)
times = 0;

View File

@ -164,7 +164,7 @@ private:
if (isDateTime64(time_column_type))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDateTime64>(time_column_col);
const auto * time_column_vec = checkAndGetColumn<ColumnDateTime64>(&time_column_col);
auto scale = assert_cast<const DataTypeDateTime64 &>(time_column_type).getScale();
if (time_column_vec)
@ -172,13 +172,13 @@ private:
}
else if (isDateTime(time_column_type))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDateTime>(time_column_col);
const auto * time_column_vec = checkAndGetColumn<ColumnDateTime>(&time_column_col);
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone);
}
else if (isDate(time_column_type))
{
const auto * time_column_vec = checkAndGetColumn<ColumnDate>(time_column_col);
const auto * time_column_vec = checkAndGetColumn<ColumnDate>(&time_column_col);
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDate &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone);
}

View File

@ -98,23 +98,23 @@ public:
static const auto uint64_data_type = std::make_shared<DataTypeNumber<UInt64>>();
auto column_successes_x = castColumnAccurate(arguments[0], uint64_data_type);
const auto & data_successes_x = checkAndGetColumn<ColumnVector<UInt64>>(column_successes_x.get())->getData();
const auto & data_successes_x = checkAndGetColumn<ColumnVector<UInt64>>(*column_successes_x).getData();
auto column_successes_y = castColumnAccurate(arguments[1], uint64_data_type);
const auto & data_successes_y = checkAndGetColumn<ColumnVector<UInt64>>(column_successes_y.get())->getData();
const auto & data_successes_y = checkAndGetColumn<ColumnVector<UInt64>>(*column_successes_y).getData();
auto column_trials_x = castColumnAccurate(arguments[2], uint64_data_type);
const auto & data_trials_x = checkAndGetColumn<ColumnVector<UInt64>>(column_trials_x.get())->getData();
const auto & data_trials_x = checkAndGetColumn<ColumnVector<UInt64>>(*column_trials_x).getData();
auto column_trials_y = castColumnAccurate(arguments[3], uint64_data_type);
const auto & data_trials_y = checkAndGetColumn<ColumnVector<UInt64>>(column_trials_y.get())->getData();
const auto & data_trials_y = checkAndGetColumn<ColumnVector<UInt64>>(*column_trials_y).getData();
static const auto float64_data_type = std::make_shared<DataTypeNumber<Float64>>();
auto column_confidence_level = castColumnAccurate(arguments[4], float64_data_type);
const auto & data_confidence_level = checkAndGetColumn<ColumnVector<Float64>>(column_confidence_level.get())->getData();
const auto & data_confidence_level = checkAndGetColumn<ColumnVector<Float64>>(*column_confidence_level).getData();
String usevar = checkAndGetColumnConst<ColumnString>(arguments[5].column.get())->getValue<String>();
String usevar = checkAndGetColumnConst<ColumnString>(*arguments[5].column).getValue<String>();
if (usevar != UNPOOLED && usevar != POOLED)
throw Exception{ErrorCodes::BAD_ARGUMENTS,

View File

@ -108,7 +108,7 @@ struct BloomFilterHash
{
const auto * array_col = typeid_cast<const ColumnArray *>(column.get());
if (checkAndGetColumn<ColumnNullable>(array_col->getData()))
if (checkAndGetColumn<ColumnNullable>(&array_col->getData()))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type {} of bloom filter index.", data_type->getName());
const auto & offsets = array_col->getOffsets();

View File

@ -219,7 +219,7 @@ static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nulla
{
/// We have to replace values masked by NULLs with defaults.
if (column.column)
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(*column.column))
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(&*column.column))
column.column = JoinCommon::filterWithBlanks(column.column, nullable_column->getNullMapColumn().getData(), true);
JoinCommon::removeColumnNullability(column);

View File

@ -334,10 +334,10 @@ public:
if ((columns.size() != 3 && columns.size() != 5) || column_position_to_check >= columns.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong number of columns: {}, position {}", columns.size(), column_position_to_check);
const auto * col = checkAndGetColumn<ColumnUInt8>(columns[column_position_to_check].get());
for (size_t i = 0; i < col->size(); ++i)
const auto & col = checkAndGetColumn<ColumnUInt8>(*columns[column_position_to_check]);
for (size_t i = 0; i < col.size(); ++i)
{
if (col->getElement(i) == 0)
if (col.getElement(i) == 0)
{
result_value = 0;
return;

View File

@ -162,7 +162,7 @@ static ColumnPtr tryConvertColumnToNullable(ColumnPtr col)
return col_lc.cloneNullable();
}
}
else if (const ColumnConst * col_const = checkAndGetColumn<ColumnConst>(*col))
else if (const ColumnConst * col_const = checkAndGetColumn<ColumnConst>(&*col))
{
const auto & nested = col_const->getDataColumnPtr();
if (nested->isNullable() || nested->canBeInsideNullable())
@ -232,7 +232,7 @@ void removeColumnNullability(ColumnWithTypeAndName & column)
if (column.column && column.column->isNullable())
{
column.column = column.column->convertToFullColumnIfConst();
const auto * nullable_col = checkAndGetColumn<ColumnNullable>(*column.column);
const auto * nullable_col = checkAndGetColumn<ColumnNullable>(column.column.get());
if (!nullable_col)
{
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Column '{}' is expected to be nullable", column.dumpStructure());
@ -258,11 +258,11 @@ void changeColumnRepresentation(const ColumnPtr & src_column, ColumnPtr & dst_co
if (nullable_src && !nullable_dst)
{
const auto * nullable = checkAndGetColumn<ColumnNullable>(*src_column);
const auto & nullable = checkAndGetColumn<ColumnNullable>(*src_column);
if (change_lowcard)
dst_column = changeLowCardinality(nullable->getNestedColumnPtr(), dst_column);
dst_column = changeLowCardinality(nullable.getNestedColumnPtr(), dst_column);
else
dst_column = nullable->getNestedColumnPtr();
dst_column = nullable.getNestedColumnPtr();
}
else if (!nullable_src && nullable_dst)
{
@ -275,7 +275,7 @@ void changeColumnRepresentation(const ColumnPtr & src_column, ColumnPtr & dst_co
{
if (change_lowcard)
{
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*src_column))
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(&*src_column))
{
dst_column = makeNullable(changeLowCardinality(nullable->getNestedColumnPtr(), dst_not_null));
assert_cast<ColumnNullable &>(*dst_column->assumeMutable()).applyNullMap(nullable->getNullMapColumn());
@ -291,7 +291,7 @@ void changeColumnRepresentation(const ColumnPtr & src_column, ColumnPtr & dst_co
ColumnPtr emptyNotNullableClone(const ColumnPtr & column)
{
if (column->isNullable())
return checkAndGetColumn<ColumnNullable>(*column)->getNestedColumnPtr()->cloneEmpty();
return checkAndGetColumn<ColumnNullable>(*column).getNestedColumnPtr()->cloneEmpty();
return column->cloneEmpty();
}
@ -374,10 +374,10 @@ ColumnRawPtrs extractKeysForJoin(const Block & block_keys, const Names & key_nam
key_columns[i] = block_keys.getByName(column_name).column.get();
/// We will join only keys, where all components are not NULL.
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*key_columns[i]))
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(&*key_columns[i]))
key_columns[i] = &nullable->getNestedColumn();
if (const auto * sparse = checkAndGetColumn<ColumnSparse>(*key_columns[i]))
if (const auto * sparse = checkAndGetColumn<ColumnSparse>(&*key_columns[i]))
key_columns[i] = &sparse->getValuesColumn();
}
@ -490,7 +490,7 @@ JoinMask getColumnAsMask(const Block & block, const String & column_name)
if (isNothing(col_type))
return JoinMask(false, block.rows());
if (const auto * const_cond = checkAndGetColumn<ColumnConst>(*src_col.column))
if (const auto * const_cond = checkAndGetColumn<ColumnConst>(&*src_col.column))
{
return JoinMask(const_cond->getBool(0), block.rows());
}

View File

@ -80,8 +80,8 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column,
if constexpr (has_left_nulls && has_right_nulls)
{
const auto * left_nullable = checkAndGetColumn<ColumnNullable>(left_column);
const auto * right_nullable = checkAndGetColumn<ColumnNullable>(right_column);
const auto * left_nullable = checkAndGetColumn<ColumnNullable>(&left_column);
const auto * right_nullable = checkAndGetColumn<ColumnNullable>(&right_column);
if (left_nullable && right_nullable)
{
@ -99,7 +99,7 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column,
if constexpr (has_left_nulls)
{
if (const auto * left_nullable = checkAndGetColumn<ColumnNullable>(left_column))
if (const auto * left_nullable = checkAndGetColumn<ColumnNullable>(&left_column))
{
if (left_column.isNullAt(lhs_pos))
return null_direction_hint;
@ -109,7 +109,7 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column,
if constexpr (has_right_nulls)
{
if (const auto * right_nullable = checkAndGetColumn<ColumnNullable>(right_column))
if (const auto * right_nullable = checkAndGetColumn<ColumnNullable>(&right_column))
{
if (right_column.isNullAt(rhs_pos))
return -null_direction_hint;

View File

@ -12,7 +12,7 @@ ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullM
if (key_columns.size() == 1)
{
auto & column = key_columns[0];
if (const auto * column_nullable = checkAndGetColumn<ColumnNullable>(*column))
if (const auto * column_nullable = checkAndGetColumn<ColumnNullable>(&*column))
{
null_map_holder = column_nullable->getNullMapColumnPtr();
null_map = &column_nullable->getNullMapData();
@ -23,7 +23,7 @@ ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullM
{
for (auto & column : key_columns)
{
if (const auto * column_nullable = checkAndGetColumn<ColumnNullable>(*column))
if (const auto * column_nullable = checkAndGetColumn<ColumnNullable>(&*column))
{
column = &column_nullable->getNestedColumn();

View File

@ -74,7 +74,7 @@ typename SetVariantsTemplate<Variant>::Type SetVariantsTemplate<Variant>::choose
for (const auto & col : key_columns)
{
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*col))
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(&*col))
{
nested_key_columns.push_back(&nullable->getNestedColumn());
has_nullable_key = true;

View File

@ -80,7 +80,7 @@ protected:
for (const auto & col : key_columns)
{
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*col))
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(&*col))
{
actual_columns.push_back(&nullable->getNestedColumn());
null_maps.push_back(&nullable->getNullMapColumn());

View File

@ -57,7 +57,7 @@ void CheckConstraintsTransform::onConsume(Chunk chunk)
auto result_column = res_column.column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality();
if (const auto * column_nullable = checkAndGetColumn<ColumnNullable>(*result_column))
if (const auto * column_nullable = checkAndGetColumn<ColumnNullable>(&*result_column))
{
const auto & nested_column = column_nullable->getNestedColumnPtr();

View File

@ -48,8 +48,8 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column,
{
if constexpr (has_left_nulls && has_right_nulls)
{
const auto * left_nullable = checkAndGetColumn<ColumnNullable>(left_column);
const auto * right_nullable = checkAndGetColumn<ColumnNullable>(right_column);
const auto * left_nullable = checkAndGetColumn<ColumnNullable>(&left_column);
const auto * right_nullable = checkAndGetColumn<ColumnNullable>(&right_column);
if (left_nullable && right_nullable)
{
@ -67,7 +67,7 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column,
if constexpr (has_left_nulls)
{
if (const auto * left_nullable = checkAndGetColumn<ColumnNullable>(left_column))
if (const auto * left_nullable = checkAndGetColumn<ColumnNullable>(&left_column))
{
if (left_nullable->isNullAt(lhs_pos))
return null_direction_hint;
@ -77,7 +77,7 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column,
if constexpr (has_right_nulls)
{
if (const auto * right_nullable = checkAndGetColumn<ColumnNullable>(right_column))
if (const auto * right_nullable = checkAndGetColumn<ColumnNullable>(&right_column))
{
if (right_nullable->isNullAt(rhs_pos))
return -null_direction_hint;

View File

@ -2516,7 +2516,7 @@ struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction
if (ts_scale_multiplier)
{
const auto & column = transform->blockAt(transform->current_row.block).input_columns[workspace.argument_column_indices[ARGUMENT_TIMESTAMP]];
const auto & curr_timestamp = checkAndGetColumn<DataTypeDateTime64::ColumnType>(column.get())->getInt(transform->current_row.row);
const auto & curr_timestamp = checkAndGetColumn<DataTypeDateTime64::ColumnType>(*column).getInt(transform->current_row.row);
Float64 time_elapsed = curr_timestamp - state.previous_timestamp;
result = (time_elapsed > 0) ? (metric_diff * ts_scale_multiplier / time_elapsed * interval_duration) : 0;

View File

@ -1101,6 +1101,10 @@ SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataP
if (is_partitioned_implementation)
{
String path = current_uri.substr(current_uri.find('/', current_uri.find("//") + 2));
if (PartitionedSink::replaceWildcards(path, "").find_first_of("*?{") != std::string::npos)
throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "URI '{}' contains globs, so the table is in readonly mode", uris.back());
return std::make_shared<PartitionedHDFSSink>(
partition_by_ast,
current_uri,

View File

@ -95,11 +95,20 @@ UInt32 DataPartStorageOnDiskFull::getRefCount(const String & file_name) const
return volume->getDisk()->getRefCount(fs::path(root_path) / part_dir / file_name);
}
std::string DataPartStorageOnDiskFull::getRemotePath(const std::string & file_name) const
std::string DataPartStorageOnDiskFull::getRemotePath(const std::string & file_name, bool if_exists) const
{
auto objects = volume->getDisk()->getStorageObjects(fs::path(root_path) / part_dir / file_name);
const std::string path = fs::path(root_path) / part_dir / file_name;
auto objects = volume->getDisk()->getStorageObjects(path);
if (objects.empty() && if_exists)
return "";
if (objects.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "One file must be mapped to one object on blob storage in MergeTree tables");
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"One file must be mapped to one object on blob storage by path {} in MergeTree tables, have {}.",
path, objects.size());
}
return objects[0].remote_path;
}

View File

@ -23,7 +23,7 @@ public:
Poco::Timestamp getFileLastModified(const String & file_name) const override;
size_t getFileSize(const std::string & file_name) const override;
UInt32 getRefCount(const std::string & file_name) const override;
std::string getRemotePath(const std::string & file_name) const override;
std::string getRemotePath(const std::string & file_name, bool if_exists) const override;
String getUniqueId() const override;
std::unique_ptr<ReadBufferFromFileBase> readFile(

View File

@ -126,7 +126,7 @@ public:
virtual UInt32 getRefCount(const std::string & file_name) const = 0;
/// Get path on remote filesystem from file name on local filesystem.
virtual std::string getRemotePath(const std::string & file_name) const = 0;
virtual std::string getRemotePath(const std::string & file_name, bool if_exists) const = 0;
virtual UInt64 calculateTotalSizeOnDisk() const = 0;

View File

@ -264,7 +264,8 @@ void IMergeTreeReader::performRequiredConversions(Columns & res_columns) const
/// Move columns from block.
name_and_type = requested_columns.begin();
for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type)
res_columns[pos] = std::move(copy_block.getByName(name_and_type->name).column);
if (copy_block.has(name_and_type->name))
res_columns[pos] = std::move(copy_block.getByName(name_and_type->name).column);
}
catch (Exception & e)
{

View File

@ -5361,20 +5361,50 @@ MergeTreeData::PartsBackupEntries MergeTreeData::backupParts(
&temp_dirs,
false, false);
auto projection_parts = part->getProjectionParts();
for (const auto & [projection_name, projection_part] : projection_parts)
auto backup_projection = [&](IDataPartStorage & storage, IMergeTreeDataPart & projection_part)
{
projection_part->getDataPartStorage().backup(
projection_part->checksums,
projection_part->getFileNamesWithoutChecksums(),
storage.backup(
projection_part.checksums,
projection_part.getFileNamesWithoutChecksums(),
fs::path{data_path_in_backup} / part->name,
backup_settings,
read_settings,
make_temporary_hard_links,
backup_entries_from_part,
&temp_dirs,
projection_part->is_broken,
projection_part.is_broken,
backup_settings.allow_backup_broken_projections);
};
auto projection_parts = part->getProjectionParts();
std::string proj_suffix = ".proj";
std::unordered_set<String> defined_projections;
for (const auto & [projection_name, projection_part] : projection_parts)
{
defined_projections.emplace(projection_name);
backup_projection(projection_part->getDataPartStorage(), *projection_part);
}
/// It is possible that the part has a written but not loaded projection,
/// e.g. it is written to parent part's checksums.txt and exists on disk,
/// but does not exist in table's projections definition.
/// Such a part can appear server was restarted after DROP PROJECTION but before old part was removed.
/// In this case, the old part will load only projections from metadata.
/// See 031145_non_loaded_projection_backup.sh.
for (const auto & [name, _] : part->checksums.files)
{
auto projection_name = fs::path(name).stem().string();
if (endsWith(name, proj_suffix) && !defined_projections.contains(projection_name))
{
auto projection_storage = part->getDataPartStorage().getProjection(projection_name + proj_suffix);
if (projection_storage->exists("checksums.txt"))
{
auto projection_part = const_cast<IMergeTreeDataPart &>(*part).getProjectionPartBuilder(
projection_name, /* is_temp_projection */false).withPartFormatFromDisk().build();
backup_projection(projection_part->getDataPartStorage(), *projection_part);
}
}
}
if (hold_storage_and_part_ptrs)

View File

@ -865,8 +865,8 @@ void MergeTreeIndexAggregatorBloomFilter::update(const Block & block, size_t * p
const auto & column_and_type = block.getByName(index_columns_name[column]);
auto index_column = BloomFilterHash::hashWithColumn(column_and_type.type, column_and_type.column, *pos, max_read_rows);
const auto & index_col = checkAndGetColumn<ColumnUInt64>(index_column.get());
const auto & index_data = index_col->getData();
const auto & index_col = checkAndGetColumn<ColumnUInt64>(*index_column);
const auto & index_data = index_col.getData();
for (const auto & hash: index_data)
column_hashes[column].insert(hash);
}

View File

@ -305,7 +305,7 @@ bool MergeTreeIndexConditionSet::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx
const NullMap * null_map = nullptr;
if (const auto * col_nullable = checkAndGetColumn<ColumnNullable>(*column))
if (const auto * col_nullable = checkAndGetColumn<ColumnNullable>(&*column))
{
col_uint8 = typeid_cast<const ColumnUInt8 *>(&col_nullable->getNestedColumn());
null_map = &col_nullable->getNullMapData();

View File

@ -46,49 +46,30 @@ bool MergeTreePrefetchedReadPool::TaskHolder::operator<(const TaskHolder & other
}
MergeTreePrefetchedReadPool::PrefetchedReaders::~PrefetchedReaders()
{
for (auto & prefetch_future : prefetch_futures)
if (prefetch_future.valid())
prefetch_future.wait();
}
MergeTreePrefetchedReadPool::PrefetchedReaders::PrefetchedReaders(
ThreadPool & pool,
MergeTreeReadTask::Readers readers_,
Priority priority_,
MergeTreePrefetchedReadPool & pool_)
MergeTreePrefetchedReadPool & read_prefetch)
: is_valid(true)
, readers(std::move(readers_))
, prefetch_runner(pool, "ReadPrepare")
{
try
prefetch_runner(read_prefetch.createPrefetchedTask(readers.main.get(), priority_));
for (const auto & reader : readers.prewhere)
prefetch_runner(read_prefetch.createPrefetchedTask(reader.get(), priority_));
fiu_do_on(FailPoints::prefetched_reader_pool_failpoint,
{
prefetch_futures.reserve(1 + readers.prewhere.size());
prefetch_futures.push_back(pool_.createPrefetchedFuture(readers.main.get(), priority_));
for (const auto & reader : readers.prewhere)
prefetch_futures.push_back(pool_.createPrefetchedFuture(reader.get(), priority_));
fiu_do_on(FailPoints::prefetched_reader_pool_failpoint,
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failpoint for prefetched reader enabled");
});
}
catch (...) /// in case of memory exceptions we have to wait
{
for (auto & prefetch_future : prefetch_futures)
if (prefetch_future.valid())
prefetch_future.wait();
throw;
}
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failpoint for prefetched reader enabled");
});
}
void MergeTreePrefetchedReadPool::PrefetchedReaders::wait()
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::WaitPrefetchTaskMicroseconds);
for (auto & prefetch_future : prefetch_futures)
prefetch_future.wait();
prefetch_runner.waitForAllToFinish();
}
MergeTreeReadTask::Readers MergeTreePrefetchedReadPool::PrefetchedReaders::get()
@ -96,13 +77,7 @@ MergeTreeReadTask::Readers MergeTreePrefetchedReadPool::PrefetchedReaders::get()
SCOPE_EXIT({ is_valid = false; });
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::WaitPrefetchTaskMicroseconds);
/// First wait for completion of all futures.
for (auto & prefetch_future : prefetch_futures)
prefetch_future.wait();
/// Then rethrow first exception if any.
for (auto & prefetch_future : prefetch_futures)
prefetch_future.get();
prefetch_runner.waitForAllToFinishAndRethrowFirstError();
return std::move(readers);
}
@ -139,7 +114,7 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool(
fillPerThreadTasks(pool_settings.threads, pool_settings.sum_marks);
}
std::future<void> MergeTreePrefetchedReadPool::createPrefetchedFuture(IMergeTreeReader * reader, Priority priority)
std::function<void()> MergeTreePrefetchedReadPool::createPrefetchedTask(IMergeTreeReader * reader, Priority priority)
{
/// In order to make a prefetch we need to wait for marks to be loaded. But we just created
/// a reader (which starts loading marks in its constructor), then if we do prefetch right
@ -147,14 +122,12 @@ std::future<void> MergeTreePrefetchedReadPool::createPrefetchedFuture(IMergeTree
/// only inside this MergeTreePrefetchedReadPool, where read tasks are created and distributed,
/// and we cannot block either, therefore make prefetch inside the pool and put the future
/// into the thread task. When a thread calls getTask(), it will wait for it is not ready yet.
auto task = [=, context = getContext()]() mutable
return [=, context = getContext()]() mutable
{
/// For async read metrics in system.query_log.
PrefetchIncrement watch(context->getAsyncReadCounters());
reader->prefetchBeginOfRange(priority);
};
return scheduleFromThreadPoolUnsafe<void>(std::move(task), prefetch_threadpool, "ReadPrepare", priority);
}
void MergeTreePrefetchedReadPool::createPrefetchedReadersForTask(ThreadTask & task)
@ -164,7 +137,7 @@ void MergeTreePrefetchedReadPool::createPrefetchedReadersForTask(ThreadTask & ta
auto extras = getExtras();
auto readers = MergeTreeReadTask::createReaders(task.read_info, extras, task.ranges);
task.readers_future = std::make_unique<PrefetchedReaders>(std::move(readers), task.priority, *this);
task.readers_future = std::make_unique<PrefetchedReaders>(prefetch_threadpool, std::move(readers), task.priority, *this);
}
void MergeTreePrefetchedReadPool::startPrefetches()

View File

@ -1,5 +1,6 @@
#pragma once
#include <Storages/MergeTree/MergeTreeReadPoolBase.h>
#include <Common/threadPoolCallbackRunner.h>
#include <Common/ThreadPool_fwd.h>
#include <IO/AsyncReadCounters.h>
#include <boost/heap/priority_queue.hpp>
@ -51,18 +52,18 @@ private:
class PrefetchedReaders
{
public:
PrefetchedReaders() = default;
PrefetchedReaders(MergeTreeReadTask::Readers readers_, Priority priority_, MergeTreePrefetchedReadPool & pool_);
PrefetchedReaders(
ThreadPool & pool, MergeTreeReadTask::Readers readers_, Priority priority_, MergeTreePrefetchedReadPool & read_prefetch);
void wait();
MergeTreeReadTask::Readers get();
bool valid() const { return is_valid; }
~PrefetchedReaders();
private:
bool is_valid = false;
MergeTreeReadTask::Readers readers;
std::vector<std::future<void>> prefetch_futures;
ThreadPoolCallbackRunnerLocal<void> prefetch_runner;
};
struct ThreadTask
@ -108,7 +109,7 @@ private:
void startPrefetches();
void createPrefetchedReadersForTask(ThreadTask & task);
std::future<void> createPrefetchedFuture(IMergeTreeReader * reader, Priority priority);
std::function<void()> createPrefetchedTask(IMergeTreeReader * reader, Priority priority);
MergeTreeReadTaskPtr stealTask(size_t thread, MergeTreeReadTask * previous_task);
MergeTreeReadTaskPtr createTask(ThreadTask & thread_task, MergeTreeReadTask * previous_task);

View File

@ -1007,6 +1007,10 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
filterColumns(columns, read_result.final_filter);
}
/// If columns not empty, then apply on-fly alter conversions if any required
if (!prewhere_info || prewhere_info->perform_alter_conversions)
merge_tree_reader->performRequiredConversions(columns);
/// If some columns absent in part, then evaluate default values
if (should_evaluate_missing_defaults)
{
@ -1017,10 +1021,6 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
addDummyColumnWithRowCount(additional_columns, read_result.num_rows);
merge_tree_reader->evaluateMissingDefaults(additional_columns, columns);
}
/// If columns not empty, then apply on-fly alter conversions if any required
if (!prewhere_info || prewhere_info->perform_alter_conversions)
merge_tree_reader->performRequiredConversions(columns);
}
read_result.columns.reserve(read_result.columns.size() + columns.size());
@ -1046,14 +1046,14 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
bool should_evaluate_missing_defaults;
merge_tree_reader->fillMissingColumns(columns, should_evaluate_missing_defaults, read_result.num_rows);
/// If some columns absent in part, then evaluate default values
if (should_evaluate_missing_defaults)
merge_tree_reader->evaluateMissingDefaults({}, columns);
/// If result not empty, then apply on-fly alter conversions if any required
if (!prewhere_info || prewhere_info->perform_alter_conversions)
merge_tree_reader->performRequiredConversions(columns);
/// If some columns absent in part, then evaluate default values
if (should_evaluate_missing_defaults)
merge_tree_reader->evaluateMissingDefaults({}, columns);
for (size_t i = 0; i < columns.size(); ++i)
read_result.columns[i] = std::move(columns[i]);
}

View File

@ -53,6 +53,19 @@ void MergeTreeReadPoolBase::fillPerPartInfos()
MergeTreeReadTaskInfo read_task_info;
read_task_info.data_part = part_with_ranges.data_part;
const auto & data_part = read_task_info.data_part;
if (data_part->isProjectionPart())
{
read_task_info.parent_part = data_part->storage.getPartIfExists(
data_part->getParentPartName(),
{MergeTreeDataPartState::PreActive, MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated});
if (!read_task_info.parent_part)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Did not find parent part {} for projection part {}",
data_part->getParentPartName(), data_part->getDataPartStorage().getFullPath());
}
read_task_info.part_index_in_query = part_with_ranges.part_index_in_query;
read_task_info.alter_conversions = part_with_ranges.alter_conversions;

View File

@ -56,6 +56,8 @@ struct MergeTreeReadTaskInfo
{
/// Data part which should be read while performing this task
DataPartPtr data_part;
/// Parent part of the projection part
DataPartPtr parent_part;
/// For `part_index` virtual column
size_t part_index_in_query;
/// Alter converversionss that should be applied on-fly for part.

View File

@ -253,11 +253,11 @@ try
bool should_evaluate_missing_defaults = false;
reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_read);
reader->performRequiredConversions(columns);
if (should_evaluate_missing_defaults)
reader->evaluateMissingDefaults({}, columns);
reader->performRequiredConversions(columns);
/// Reorder columns and fill result block.
size_t num_columns = sample.size();
Columns res_columns;

View File

@ -377,7 +377,16 @@ IMergeTreeDataPart::Checksums checkDataPart(
auto file_name = it->name();
if (!data_part_storage.isDirectory(file_name))
{
auto remote_path = data_part_storage.getRemotePath(file_name);
const bool is_projection_part = data_part->isProjectionPart();
auto remote_path = data_part_storage.getRemotePath(file_name, /* if_exists */is_projection_part);
if (remote_path.empty())
{
chassert(is_projection_part);
throw Exception(
ErrorCodes::BROKEN_PROJECTION,
"Remote path for {} does not exist for projection path. Projection {} is broken",
file_name, data_part->name);
}
cache.removePathIfExists(remote_path, FileCache::getCommonUser().user_id);
}
}

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